diff --git a/CHANGES.txt b/CHANGES.txt index e70fc6e23b55..2c40009a498b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,12 @@ 5.0.7 + * Improved observability in AutoRepair to report both expected vs. actual repair bytes and expected vs. actual keyspaces (CASSANDRA-20581) + * Stop repair scheduler if two major versions are detected (CASSANDRA-20048) + * AutoRepair: Safeguard Full repair against disk protection (CASSANDRA-20045) + * Stop AutoRepair monitoring thread upon Cassandra shutdown (CASSANDRA-20623) + * Fix race condition in auto-repair scheduler (CASSANDRA-20265) + * Implement minimum repair task duration setting for auto-repair scheduler (CASSANDRA-20160) + * Implement preview_repaired auto-repair type (CASSANDRA-20046) + * Automated Repair Inside Cassandra for CEP-37 (CASSANDRA-19918) * Automatically disable zero-copy streaming for legacy sstables with old bloom filter format (CASSANDRA-21092) * Fix CQLSSTableWriter serialization of vector of date and time (CASSANDRA-20979) * Correctly calculate default for FailureDetector max interval (CASSANDRA-21025) diff --git a/NEWS.txt b/NEWS.txt index 1192f4178a5d..c0e53dcb0eb4 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -74,6 +74,23 @@ Upgrading who did not use LZ4 native libraries, this will now fallback to a safer but less performant pure Java implementation. During startup, a warning will be logged if the LZ4 native library is not available. + - The auto-repair feature introduced in 5.0.7 requires enabling the JVM property + `cassandra.autorepair.enable=true` (add `-Dcassandra.autorepair.enable=true` to JVM options) before starting + the node. This property creates the required schema elements for auto-repair, including the auto_repair column + in system_schema.tables and system_schema.views, as well as the auto_repair_history and auto_repair_priority + tables in system_distributed. After enabling this property, you still need to enable auto-repair scheduling + either in cassandra.yaml under the `auto_repair` section or at runtime via JMX. + + Users who do not intend to use auto-repair can leave this property disabled (the default) to maintain schema + compatibility with pre-5.0.7 nodes during rolling upgrades. This property must be set consistently across all + nodes before startup and cannot be changed at runtime. + + WARNING: This property is non-reversible. Once enabled, it cannot be disabled. Attempting to start a node + with `cassandra.autorepair.enable=false` after it was previously enabled will cause the node to fail during + initialization due to schema incompatibility (the persisted schema contains auto-repair columns that are not + recognized when the property is disabled). To disable auto-repair scheduling after the property has been + enabled, use cassandra.yaml or JMX instead of changing the JVM property. + 5.0.5 ===== @@ -81,6 +98,12 @@ New features ------------ - Full support for Java 17, it is not experimental anymore. + - CEP-37 Auto Repair is a fully automated scheduler that provides repair orchestration within Apache Cassandra. This + significantly reduces operational overhead by eliminating the need for operators to deploy external tools to submit + and manage repairs. See + https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-37+Apache+Cassandra+Unified+Repair+Solution for more + details on the motivation and design. + 5.0.4 ===== diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index e09518188a60..0f17bb2032b1 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1951,6 +1951,13 @@ report_unconfirmed_repaired_data_mismatches: false # Materialized views are considered experimental and are not recommended for production use. materialized_views_enabled: false +# Specify whether Materialized View mutations are replayed through the write path on streaming, e.g. repair. +# When enabled, Materialized View data streamed to the destination node will be written into commit log first. When setting to false, +# the streamed Materialized View data is written into SSTables just the same as normal streaming. The default is true. +# If this is set to false, streaming will be considerably faster however it's possible that, in extreme situations +# (losing > quorum # nodes in a replica set), you may have data in your SSTables that never makes it to the Materialized View. +# materialized_views_on_repair_enabled: true + # Enables SASI index creation on this node. # SASI indexes are considered experimental and are not recommended for production use. sasi_indexes_enabled: false @@ -2253,6 +2260,7 @@ drop_compact_storage_enabled: false # excluded_keyspaces: # comma separated list of keyspaces to exclude from the check # excluded_tables: # comma separated list of keyspace.table pairs to exclude from the check + # This property indicates with what Cassandra major version the storage format will be compatible with. # # The chosen storage compatibility mode will determine the versions of the written sstables, commitlogs, hints, etc. @@ -2281,3 +2289,170 @@ drop_compact_storage_enabled: false # compatibility mode would no longer toggle behaviors as when it was running in the UPGRADING mode. # storage_compatibility_mode: CASSANDRA_4 + + +# Prevents preparing a repair session or beginning a repair streaming session if pending compactions is over +# the given value. Defaults to disabled. +# reject_repair_compaction_threshold: 1024 + +# At least 20% of disk must be unused to run repair. It is useful to avoid disks filling up during +# repair as anti-compaction during repair may contribute to additional space temporarily. +# if you want to disable this feature (the recommendation is not to, but if you want to disable it for whatever reason) +# then set the ratio to 0.0 +# repair_disk_headroom_reject_ratio: 0.2; + +# This is the deprecated config which was used to safeguard incremental repairs. Use repair_disk_headroom_reject_ratio +# instead as it safeguards against all repairs. +# incremental_repair_disk_headroom_reject_ratio: 0.2; + +# Configuration for Auto Repair Scheduler. +# +# This feature is disabled by default. +# +# See: https://cassandra.apache.org/doc/latest/cassandra/managing/operating/auto_repair.html for an overview of this +# feature. +# +# auto_repair: +# # Enable/Disable the auto-repair scheduler. +# # If set to false, the scheduler thread will not be started. +# # If set to true, the repair scheduler thread will be created. The thread will +# # check for secondary configuration available for each repair type (full, incremental, +# # and preview_repaired), and based on that, it will schedule repairs. +# enabled: true +# repair_type_overrides: +# full: +# # Enable/Disable full auto-repair +# enabled: true +# # Minimum duration between repairing the same node again. This is useful for tiny clusters, +# # such as clusters with 5 nodes that finish repairs quickly. This means that if the scheduler completes one +# # round on all nodes in less than this duration, it will not start a new repair round on a given node until +# # this much time has passed since the last repair completed. Consider increasing to a larger value to reduce +# # the impact of repairs, however note that one should attempt to run repairs at a smaller interval than +# # gc_grace_seconds to avoid potential data resurrection. +# min_repair_interval: 24h +# token_range_splitter: +# # Implementation of IAutoRepairTokenRangeSplitter; responsible for splitting token ranges +# # for repair assignments. +# # +# # Out of the box, Cassandra provides org.apache.cassandra.repair.autorepair.{RepairTokenRangeSplitter, +# # FixedTokenRangeSplitter}. +# # +# # - RepairTokenRangeSplitter (default) attempts to intelligently split ranges based on data size and partition +# # count. +# # - FixedTokenRangeSplitter splits into fixed ranges based on the 'number_of_subranges' option. +# # class_name: org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter +# +# # Optional parameters can be specified in the form of: +# # parameters: +# # param_key1: param_value1 +# parameters: +# # The target and maximum amount of compressed bytes that should be included in a repair assignment. +# # This scopes the amount of work involved in a repair and includes the data covering the range being +# # repaired. +# bytes_per_assignment: 50GiB +# # The maximum number of bytes to cover in an individual schedule. This serves as +# # a mechanism to throttle the work done in each repair cycle. You may reduce this +# # value if the impact of repairs is causing too much load on the cluster or increase it +# # if writes outpace the amount of data being repaired. Alternatively, adjust the +# # min_repair_interval. +# # This is set to a large value for full repair to attempt to repair all data per repair schedule. +# max_bytes_per_schedule: 100000GiB +# incremental: +# enabled: false +# # Incremental repairs operate over unrepaired data and should finish quickly. Running incremental repair +# # frequently keeps the unrepaired set smaller and thus causes repairs to operate over a smaller set of data, +# # so a more frequent schedule such as 1h is recommended. +# # NOTE: Please consult +# # https://cassandra.apache.org/doc/latest/cassandra/managing/operating/auto_repair.html#enabling-ir +# # for guidance on enabling incremental repair on ane exiting cluster. +# min_repair_interval: 24h +# token_range_splitter: +# parameters: +# # Configured to attempt repairing 50GiB of compressed data per repair. +# # This throttles the amount of incremental repair and anticompaction done per schedule after incremental +# # repairs are turned on. +# bytes_per_assignment: 50GiB +# # Restricts the maximum number of bytes to cover in an individual schedule to the configured +# # max_bytes_per_schedule value (defaults to 100GiB for incremental). +# # Consider increasing this value if more data is written than this limit within the min_repair_interval. +# max_bytes_per_schedule: 100GiB +# preview_repaired: +# # Performs preview repair over repaired SSTables, useful to detect possible inconsistencies in the repaired +# # data set. +# enabled: false +# min_repair_interval: 24h +# token_range_splitter: +# parameters: +# bytes_per_assignment: 50GiB +# max_bytes_per_schedule: 100000GiB +# # Time interval between successive checks to see if ongoing repairs are complete or if it is time to schedule +# # repairs. +# repair_check_interval: 5m +# # Minimum duration for the execution of a single repair task. This prevents the scheduler from overwhelming +# # the node by scheduling too many repair tasks in a short period of time. +# repair_task_min_duration: 5s +# # The scheduler needs to adjust its order when nodes leave the ring. Deleted hosts are tracked in metadata +# # for a specified duration to ensure they are indeed removed before adjustments are made to the schedule. +# history_clear_delete_hosts_buffer_interval: 2h +# # By default repair is disabled if there are mixed major versions detected - which would happen +# # if a major version upgrade is being performed on the cluster, but a user can enable it using this flag +# mixed_major_version_repair_enabled: false +# # NOTE: Each of the below settings can be overridden per repair type under repair_type_overrides +# global_settings: +# # If true, attempts to group tables in the same keyspace into one repair; otherwise, each table is repaired +# # individually. +# repair_by_keyspace: true +# # Number of threads to use for each repair job scheduled by the scheduler. Similar to the -j option in nodetool +# # repair. +# number_of_repair_threads: 1 +# # Number of nodes running repair in parallel. If parallel_repair_percentage is set, the larger value is used. +# parallel_repair_count: 3 +# # Percentage of nodes in the cluster running repair in parallel. If parallel_repair_count is set, the larger value +# # is used. +# parallel_repair_percentage: 3 +# # Whether to allow a node to take its turn running repair while one or more of its replicas are running repair. +# # Defaults to false, as running repairs concurrently on replicas can increase load and also cause anticompaction +# # conflicts while running incremental repair. +# allow_parallel_replica_repair: false +# # An addition to allow_parallel_replica_repair that also blocks repairs when replicas (including this node itself) +# # are repairing in any schedule. For example, if a replica is executing full repairs, a value of false will +# # prevent starting incremental repairs for this node. Defaults to true and is only evaluated when +# # allow_parallel_replica_repair is false. +# allow_parallel_replica_repair_across_schedules: true +# # Repairs materialized views if true. +# materialized_view_repair_enabled: false +# # Delay before starting repairs after a node restarts to avoid repairs starting immediately after a restart. +# initial_scheduler_delay: 5m +# # Timeout for retrying stuck repair sessions. +# repair_session_timeout: 3h +# # Force immediate repair on new nodes after they join the ring. +# force_repair_new_node: false +# # Threshold to skip repairing tables with too many SSTables. Defaults to 10,000 SSTables to avoid penalizing good +# # tables. +# sstable_upper_threshold: 50000 +# # Maximum time allowed for repairing one table on a given node. If exceeded, the repair proceeds to the +# # next table. +# table_max_repair_time: 6h +# # Avoid running repairs in specific data centers. By default, repairs run in all data centers. Specify data +# # centers to exclude in this list. Note that repair sessions will still consider all replicas from excluded +# # data centers. Useful if you have keyspaces that are not replicated in certain data centers, and you want to +# # not run repair schedule in certain data centers. +# ignore_dcs: [] +# # Repair only the primary ranges owned by a node. Equivalent to the -pr option in nodetool repair. Defaults +# # to true. General advice is to keep this true. +# repair_primary_token_range_only: true +# # Maximum number of retries for a repair session. +# repair_max_retries: 3 +# # Backoff time before retrying a repair session. +# repair_retry_backoff: 30s +# token_range_splitter: +# # Splitter implementation to generate repair assignments. Defaults to RepairTokenRangeSplitter. +# class_name: org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter +# parameters: +# # Maximum number of partitions to include in a repair assignment. Used to reduce number of partitions +# # present in merkle tree leaf nodes to avoid overstreaming. +# partitions_per_assignment: 1048576 +# # Maximum number of tables to include in a repair assignment. This reduces the number of repairs, +# # especially in keyspaces with many tables. The splitter avoids batching tables together if they +# # exceed other configuration parameters like bytes_per_assignment or partitions_per_assignment. +# max_tables_per_assignment: 64 diff --git a/doc/modules/cassandra/nav.adoc b/doc/modules/cassandra/nav.adoc index 7c1a02cfaa94..d9a028427d72 100644 --- a/doc/modules/cassandra/nav.adoc +++ b/doc/modules/cassandra/nav.adoc @@ -84,6 +84,7 @@ **** xref:cassandra:managing/configuration/cass_jvm_options_file.adoc[jvm-* files] **** xref:cassandra:managing/configuration/configuration.adoc[Liberating cassandra.yaml Parameters' Names from Their Units] *** xref:cassandra:managing/operating/index.adoc[] +**** xref:cassandra:managing/operating/auto_repair.adoc[Auto Repair] **** xref:cassandra:managing/operating/backups.adoc[Backups] **** xref:cassandra:managing/operating/bloom_filters.adoc[Bloom filters] **** xref:cassandra:managing/operating/bulk_loading.adoc[Bulk loading] @@ -125,4 +126,4 @@ *** xref:reference/static.adoc[Static columns] *** xref:reference/vector-data-type.adoc[Vector data type] -** xref:integrating/plugins/index.adoc[] \ No newline at end of file +** xref:integrating/plugins/index.adoc[] diff --git a/doc/modules/cassandra/pages/managing/operating/auto_repair.adoc b/doc/modules/cassandra/pages/managing/operating/auto_repair.adoc new file mode 100644 index 000000000000..e989c49d2a9e --- /dev/null +++ b/doc/modules/cassandra/pages/managing/operating/auto_repair.adoc @@ -0,0 +1,460 @@ += Auto Repair +:navtitle: Auto Repair +:description: Auto Repair concepts - How it works, how to configure it, and more. +:keywords: CEP-37, Repair, Incremental, Preview + +Auto Repair is a fully automated scheduler that provides repair orchestration within Apache Cassandra. This +significantly reduces operational overhead by eliminating the need for operators to deploy external tools to submit and +manage repairs. + +At a high level, a dedicated thread pool is assigned to the repair scheduler. The repair scheduler in Cassandra +maintains a new replicated table, `system_distributed.auto_repair_history`, which stores the repair history for all +nodes, including details such as the last repair time. The scheduler selects the node(s) to begin repairs and +orchestrates the process to ensure that every table and its token ranges are repaired. + +The algorithm can run repairs simultaneously on multiple nodes and splits token ranges into subranges, with necessary +retries to handle transient failures. Automatic repair starts as soon as a Cassandra cluster is launched, similar to +compaction, and if configured appropriately, does not require human intervention. + +The scheduler currently supports Full, Incremental, and Preview repair types with the following features. New repair +types, such as Paxos repair or other future repair mechanisms, can be integrated with minimal development effort! + + +== Features +- Capability to run repairs on multiple nodes simultaneously. +- A default implementation and an interface to override the dataset being repaired per session. +- Extendable token split algorithms with two implementations readily available: +. Splits token ranges by placing a cap on the size of data repaired in one session and a maximum cap at the schedule +level using xref:#repair-token-range-splitter[RepairTokenRangeSplitter] (default). +. Splits tokens evenly based on the specified number of splits using +xref:#fixed-split-token-range-splitter[FixedSplitTokenRangeSplitter]. +- A new xref:#table-configuration[CQL table property] (`auto_repair`) offering: +. The ability to disable specific repair types at the table level, allowing the scheduler to skip one or more tables. +. Configuring repair priorities for certain tables to prioritize them over others. +- Dynamic enablement or disablement of the scheduler for each repair type. +- Configurable settings tailored to each repair job. +- Rich configuration options for each repair type (e.g., Full, Incremental, or Preview repairs). +- Comprehensive observability features that allow operators to configure alarms as needed. + +== Considerations + +Before enabling Auto Repair, please consult the xref:managing/operating/repair.adoc[Repair] guide to establish a base +understanding of repairs. + +=== Full Repair + +Full Repairs operate over all data in the token range being repaired. It is therefore important to run full repair +with a longer schedule and with smaller assignments. + +=== Incremental Repair + +When enabled from the inception of a cluster, incremental repairs operate over unrepaired data and should finish +quickly when run more frequently. + +Once incremental repair has been run, SSTables will be separated between data that have been incrementally repaired +and data that have not. Therefore, it is important to continually run incremental repair once it has been enabled so +newly written data can be compacted together with previously repaired data, allowing overwritten and expired data to +be eventually purged. + +Running incremental repair more frequently keeps the unrepaired set smaller and thus causes repairs to operate over +a smaller set of data, so a shorter `min_repair_interval` such as `1h` is recommended for new clusters. + +==== Enabling Incremental Repair on existing clusters with a large amount of data +[#enabling-ir] +One should be careful when enabling incremental repair on a cluster for the first time. While +xref:#repair-token-range-splitter[RepairTokenRangeSplitter] includes a default configuration to attempt to gracefully +migrate to incremental repair over time, failure to take proper precaution could overwhelm the cluster with +xref:managing/operating/compaction/overview.adoc#types-of-compaction[anticompactions]. + +No matter how one goes about enabling and running incremental repair, it is recommended to run a cycle of full repairs +for the entire cluster as pre-flight step to running incremental repair. This will put the cluster into a more +consistent state which will reduce the amount of streaming between replicas when incremental repair initially runs. + +If you do not have strong data consistency requirements, one may consider using +xref:managing/tools/sstable/sstablerepairedset.adoc[nodetool sstablerepairedset] to mark all SSTables as repaired +before enabling incremental repair scheduling using Auto Repair. This will reduce the burden of initially running +incremental repair because all existing data will be considered as repaired, so subsequent incremental repairs will +only run against new data. + +If you do have strong data consistency requirements, then one must treat all data as initially unrepaired and run +incremental repair against it. Consult +xref:#incremental-repair-defaults[RepairTokenRangeSplitter's Incremental repair defaults]. + +In particular one should be mindful of the xref:managing/operating/compaction/overview.adoc[compaction strategy] +you use for your tables and how it might impact incremental repair before running incremental repair for the first +time: + +- *Large SSTables*: When using xref:managing/operating/compaction/stcs.adoc[SizeTieredCompactionStrategy] or any + compaction strategy which can create large SSTables including many partitions the amount of + xref:managing/operating/compaction/overview.adoc#types-of-compaction[anticompaction] that might be required could be + excessive. Using a small `bytes_per_assignment` might contribute to repeated anticompactions over the same + unrepaired data. +- *Partitions overlapping many SSTables*: If partitions overlap between many SSTables, the amount of SSTables included + in a repair might be large. Therefore it is important to consider that many SSTables may be included in a repair + session and must all be anticompacted. xref:managing/operating/compaction/lcs.adoc[LeveledCompactionStrategy] is less + susceptible to this issue as it prevents overlapping of partitions within levels outside of L0, but if SSTables + start accumulating in L0 between incremental repairs, the cost of anticompaction will increase. + xref:managing/operating/compaction/ucs#sharding[UnifiedCompactionStrategy's sharding] can also be used to avoid + partitions overlapping SSTables. + +The xref:#repair-token-range-splitter[token_range_splitter] configuration for incremental repair includes a default +configuration that attempts to conservatively migrate 100GiB of compressed data every day per node. Depending on +requirements, data set and capability of a cluster's hardware, one may consider tuning these values to be more +aggressive or conservative. + +=== Previewing Repaired Data + +The `preview_repaired` repair type executes repairs over the repaired data set to detect possible data inconsistencies. + +Inconsistencies in the repaired data set should not happen in practice and could indicate a possible bug in incremental +repair. + +Running preview repairs is useful when considering using the +xref:cassandra:managing/operating/compaction/tombstones.adoc#deletion[only_purge_repaired_tombstones] table compaction +option to prevent data from possibly being resurrected when inconsistent replicas are missing tombstones from deletes. + +When enabled, the `BytesPreviewedDesynchronized` and `TokenRangesPreviewedDesynchronized` +xref:cassandra:managing/operating/metrics.adoc#table-metrics[table metrics] can be used to detect inconsistencies in the +repaired data set. + +== Configuring Auto Repair in cassandra.yaml + +Configuration for Auto Repair is managed in the `cassandra.yaml` file by the `auto_repair` property. + +A rich set of configuration exists for configuring Auto Repair with sensible defaults. However, the expectation +is that some tuning might be needed particulary when it comes to tuning how often repair should run +(`min_repair_interval`) and how repair assignments as created (`token_range_splitter`). + +The following is a practical example of an auto_repair configuration that one might use. + +[source, yaml] +---- +auto_repair: + enabled: true + repair_type_overrides: + full: + enabled: true + min_repair_interval: 5d + incremental: + enabled: true + min_repair_interval: 1h + token_range_splitter: + parameters: + bytes_per_assignment: 50GiB + max_bytes_per_schedule: 100GiB + preview_repaired: + enabled: true + min_repair_interval: 1d + global_settings: + repair_by_keyspace: true + parallel_repair_count: 1 +---- + + +=== Top level settings +The following settings are defined at the top level of the configuration file and apply universally across all +repair types. + +[cols=",,",options="header",] +|=== +| Name | Default | Description +| enabled | false | Enable/Disable the auto-repair scheduler. If set to false, the scheduler thread will not be started. +If set to true, the repair scheduler thread will be created. The thread will check for secondary configuration available +for each repair type (full, incremental, and preview_repaired), and based on that, it will schedule repairs. +| repair_check_interval | 5m | Time interval between successive checks to see if ongoing repairs are complete or if it +is time to schedule repairs. +| repair_max_retries | 3 | Maximum number of retries for a repair session. +| history_clear_delete_hosts_buffer_interval | 2h | The scheduler needs to adjust its order when nodes leave the ring. +Deleted hosts are tracked in metadata for a specified duration to ensure they are indeed removed before adjustments +are made to the schedule. +| mixed_major_version_repair_enabled | false | Enable/Disable running repairs on the cluster when there are mixed +major versions detected, which usually occurs when the cluster is being upgraded. Repairs between nodes of +different major versions is not something that is tested, so this may lead to data compatibility issues. +It is strongly discouraged to set this to true without doing extensive testing beforehand. +|=== + + +=== Repair level settings +The following settings can be configured globally using `global_settings` or tailored individually for each repair +type by using `repair_type_overrides`. + +[cols=",,",options="header",] +|=== +| Name | Default | Description +| enabled | false | Whether the given repair types should be enabled +| min_repair_interval | 24h | Minimum duration between repairing the same node again. This is useful for tiny clusters, +such as clusters with 5 nodes that finish repairs quickly. This means that if the scheduler completes one round on all +nodes in less than this duration, it will not start a new repair round on a given node until this much time has +passed since the last repair completed. Consider increasing to a larger value to reduce the impact of repairs, +however note that one should attempt to run repairs at a smaller interval than gc_grace_seconds to +avoid xref:cassandra:managing/operating/compaction/tombstones.adoc#zombies[data resurrection]. +| token_range_splitter.class_name | org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter | Implementation of +IAutoRepairTokenRangeSplitter to use; responsible for splitting token ranges for repair assignments. Out of the box, +Cassandra provides org.apache.cassandra.repair.autorepair.{RepairTokenRangeSplitter,FixedTokenRangeSplitter}. +| repair_by_keyspace | true | If true, attempts to group tables in the same keyspace into one repair; otherwise, +each table is repaired individually. +| number_of_repair_threads | 1 | Number of threads to use for each repair job scheduled by the scheduler. Similar to +the -j option in nodetool repair. +| parallel_repair_count | 3 | Number of nodes running repair in parallel. If `parallel_repair_percentage` is set, the +larger value is used. +| parallel_repair_percentage | 3 | Percentage of nodes in the cluster running repair in parallel. If +`parallel_repair_count is set`, the larger value is used. +| allow_parallel_replica_repair | false | Whether to allow a node to take its turn running repair while one or more of +its replicas are running repair. Defaults to false, as running repairs concurrently on replicas can increase load and +also cause anticompaction conflicts while running incremental repair. +| allow_parallel_replica_repair_across_schedules | true | An addition to allow_parallel_repair that also blocks repairs +when replicas (including this node itself) are repairing in any schedule. +For example, if a replica is executing full repairs, a value of false will prevent starting incremental repairs for this +node. Defaults to true and is only evaluated when allow_parallel_replica_repair is false. +| materialized_view_repair_enabled | false | Repairs materialized views if true. +| initial_scheduler_delay | 5m | Delay before starting repairs after a node restarts to avoid repairs starting +immediately after a restart. +| repair_session_timeout | 3h | Timeout for retrying stuck repair sessions. +| force_repair_new_node | false | Force immediate repair on new nodes after they join the ring. +| sstable_upper_threshold | 50000 | Threshold to skip repairing tables with too many SSTables. +| table_max_repair_time | 6h | Maximum time allowed for repairing one table on a given node. If exceeded, the repair +proceeds to the next table. +| ignore_dcs | [] | Avoid running repairs in specific data centers. By default, repairs run in all data centers. Specify +data centers to exclude in this list. Note that repair sessions will still consider all replicas from excluded data +centers. Useful if you have keyspaces that are not replicated in certain data centers, and you want to not run repair +schedule in certain data centers. +| repair_primary_token_range_only | true | Repair only the primary ranges owned by a node. Equivalent to the -pr option +in nodetool repair. General advice is to keep this true. +| repair_retry_backoff | 30s | Backoff time before retrying a repair session. +| repair_task_min_duration | 5s | Minimum duration for the execution of a single repair task. This prevents the +scheduler from overwhelming the node by scheduling too many repair tasks in a short period of time. +|=== + +=== `RepairTokenRangeSplitter` configuration +[#repair-token-range-splitter] + +`RepairTokenRangeSplitter` is the default implementation of `IAutoRepairTokenRangeSplitter` that attempts to create +token range assignments meeting the following goals: + +- *Create smaller, consistent repair times*: Long repairs, such as those lasting 15 hours, can be problematic. If a +node fails 14 hours into the repair, the entire process must be restarted. The goal is to reduce the impact of +disturbances or failures. However, making the repairs too short can lead to overhead from repair orchestration becoming +the main bottleneck. + +- *Minimize the impact on hosts*: Repairs should not heavily affect the host systems. For incremental repairs, this +might involve anti-compaction work. In full repairs, streaming large amounts of data—especially with wide partitions +can lead to issues with disk usage and higher compaction costs. + +- *Reduce overstreaming*: The Merkle tree, which represents data within each partition and range, has a maximum size. +If a repair covers too many partitions, the tree’s leaves represent larger data ranges. Even a small change in a leaf +can trigger excessive data streaming, making the process inefficient. + +- *Reduce number of repairs*: If there are many small tables, it's beneficial to batch these tables together under a +single parent repair. This prevents the repair overhead from becoming a bottleneck, especially when dealing with +hundreds of tables. Running individual repairs for each table can significantly impact performance and efficiency. + +To achieve these goals, this implementation inspects SSTable metadata to estimate the bytes and number of partitions +within a range and splits it accordingly to bound the size of the token ranges used for repair assignments. + +==== Parameter defaults + +The following `parameters` include the same defaults for all repair types. + +[cols=",,",options="header",] +|=== +| Name | Default | Description +| partitions_per_assignment | 1048576 | Maximum number of partitions to include in a repair +assignment. Used to reduce number of partitions present in merkle tree leaf nodes to avoid overstreaming. +| max_tables_per_assignment | 64 | Maximum number of tables to include in a repair assignment. +This reduces the number of repairs, especially in keyspaces with many tables. The splitter avoids batching tables +together if they exceed other configuration parameters like `bytes_per_assignment` or `partitions_per_assignment`. +|=== + +==== Full & Preview Repaired repair defaults + +The following `parameters` defaults are established for both `full` and `preview_repaired` repair scheduling: + +[cols=",,",options="header",] +|=== +| Name | Default | Description +| bytes_per_assignment | 50GiB | The target and maximum amount of *compressed* bytes that should be included in a +repair assignment. *Note*: For full and preview_repaired, only the portion of an SSTable that covers the ranges +being repaired are accounted for in this calculation. +| max_bytes_per_schedule | 100000GiB | The maximum number of bytes to cover in an individual +schedule. This serves as a mechanism to throttle the work done in each repair cycle. You may reduce this value if the +impact of repairs is causing too much load on the cluster or increase it if writes outpace the amount of data being +repaired. Alternatively, adjust the `min_repair_interval`. This is set to a large value for full repair to attempt to +repair all data per repair schedule. +|=== + +==== Incremental repair defaults + +The following `parameters` defaults are established for `incremental` repair scheduling: + +[cols=",,",options="header",] +|=== +| Name | Default | Description +| bytes_per_assignment | 50GiB | The target and maximum amount of *compressed* bytes that should be +included in a repair assignment. *Note*: For incremental repair, the *entire size* of *unrepaired* SSTables +including ranges being repaired are accounted for in this calculation. This is to account for the anticompaction +work required to split the candidate data to repair from the data that won't be repaired. +| max_bytes_per_schedule | 100GiB | The maximum number of bytes to cover in an individual schedule. +Consider increasing if more data is written than this limit within the `min_repair_interval`. +|=== + +=== `FixedSplitTokenRangeSplitter` configuration +[#fixed-split-token-range-splitter] + +`FixedSplitTokenRangeSplitter` is a more simple implementation of `IAutoRepairTokenRangeSplitter` that creates repair +assignments by splitting a node's token ranges into an even number of splits. + +The following `parameters` apply for `FixedSplitTokenRangeSplitter` configuration: + +[cols=",,",options="header",] +|=== +| Name | Default | Description +| number_of_subranges | 32 | Number of evenly split subranges to create for each node that repair runs for. +If vnodes are configured using `num_tokens`, attempts to evenly subdivide subranges by each range. For example, for +`num_tokens: 16` and `number_of_subranges: 32`, 2 (32/16) repair assignments will be created for each token range. At +least one repair assignment will be created for each token range. +|=== + +=== Other cassandra.yaml Considerations + +==== Enable `reject_repair_compaction_threshold` + +When enabling auto_repair, it is advisable to configure the top level `reject_repair_compaction_threshold` +configuration in cassandra.yaml as a backpressure mechanism to reject new repairs on instances that have many +pending compactions. + +==== Tune `repair_disk_headroom_reject_ratio` + +By default, repairs will be rejected if less than 20% of disk is available. If one wishes to be +conservative this top level configuration could be increased to a larger value to prevent filling your data directories. + +== Table configuration + +If Auto Repair is enabled in cassandra.yaml, the `auto_repair` property may be optionally configured at the table +level, e.g.: + +[source,cql] +---- +ALTER TABLE cycling.cyclist_races +WITH auto_repair = {'incremental_enabled': 'false', 'priority': '0'}; +---- + +[cols=",,",options="header",] +|=== +| Name | Default | Description +| priority | 0 | Indicates the priority at which this table should be given when issuing repairs. The higher the number +the more priority will be given to repair the table (e.g. 3 will be repaired before 2). When `repair_by_keyspace` is +set to `true` tables sharing the same priority may be grouped in the same repair assignment. +| full_enabled | true | Whether full repair is enabled for this table. If full.enabled is not true in cassandra.yaml +this will not be evaluated. +| incremental_enabled | true | Whether incremental repair is enabled for this table. If incremental.enabled is not +true in cassandra.yaml this will not be evaluated. +| preview_repaired_enabled | true | Whether preview repair is enabled for this table. If preview_repaired.enabled is +not true in cassandra.yaml this will not be evaluated. +|=== + +== Nodetool Configuration +=== nodetool getautorepairconfig + +Retrieves the runtime configuration of Auto Repair for the targeted node. + +[source,none] +---- +$> nodetool getautorepairconfig +repair scheduler configuration: + repair_check_interval: 5m + repair_max_retries: 3 + history_clear_delete_hosts_buffer_interval: 2h +configuration for repair_type: full + enabled: true + min_repair_interval: 24h + repair_by_keyspace: true + number_of_repair_threads: 1 + sstable_upper_threshold: 50000 + table_max_repair_time: 6h + ignore_dcs: [] + repair_primary_token_range_only: true + parallel_repair_count: 3 + parallel_repair_percentage: 3 + materialized_view_repair_enabled: false + initial_scheduler_delay: 5m + repair_session_timeout: 3h + force_repair_new_node: false + repair_retry_backoff: 30s + repair_task_min_duration: 5s + token_range_splitter: org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter + token_range_splitter.bytes_per_assignment: 50GiB + token_range_splitter.partitions_per_assignment: 1048576 + token_range_splitter.max_tables_per_assignment: 64 + token_range_splitter.max_bytes_per_schedule: 100000GiB +configuration for repair_type: incremental + enabled: true + min_repair_interval: 1h + repair_by_keyspace: true + number_of_repair_threads: 1 + sstable_upper_threshold: 50000 + table_max_repair_time: 6h + ignore_dcs: [] + repair_primary_token_range_only: true + parallel_repair_count: 3 + parallel_repair_percentage: 3 + materialized_view_repair_enabled: false + initial_scheduler_delay: 5m + repair_session_timeout: 3h + force_repair_new_node: false + repair_retry_backoff: 30s + repair_task_min_duration: 5s + token_range_splitter: org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter + token_range_splitter.bytes_per_assignment: 50GiB + token_range_splitter.partitions_per_assignment: 1048576 + token_range_splitter.max_tables_per_assignment: 64 + token_range_splitter.max_bytes_per_schedule: 100GiB +configuration for repair_type: preview_repaired + enabled: false +---- + +=== nodetool autorepairstatus + +Provides currently running Auto Repair status. + +[source,none] +---- +$> nodetool autorepairstatus -t incremental +Active Repairs +425cea55-09aa-46e0-8911-9f37a4424574 + + +$> nodetool autorepairstatus -t full +Active Repairs +NONE + +---- + +=== nodetool setautorepairconfig + +Dynamic configuration changes can be made by using `setautorepairconfig`. Note that this only applies on the node being +targeted and these changes are not retained when a node is bounced. + +The following disables the `incremental` repair schedule: + +[source,none] +---- +$> nodetool setautorepairconfig -t incremental enabled false +---- + +The following adjusts the `min_repair_interval` option to `5d` specifically for the `full` repair schedule: + +[source,none] +---- +$> nodetool setautorepairconfig -t full min_repair_interval 5d +---- + +The following configures the `bytes_per_assignment` parameter for `incremental` repair's `token_range_splitter` to +`10GiB`: + +[source,none] +---- +$> nodetool setautorepairconfig -t incremental token_range_splitter.bytes_per_assignment 10GiB +---- + +==== More details +https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-37+Apache+Cassandra+Unified+Repair+Solution[CEP-37] diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index f8b983ba74c5..94c129f68d5b 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -56,6 +56,8 @@ public enum CassandraRelevantProperties AUTH_CACHE_WARMING_MAX_RETRIES("cassandra.auth_cache.warming.max_retries"), AUTH_CACHE_WARMING_RETRY_INTERVAL_MS("cassandra.auth_cache.warming.retry_interval_ms"), AUTOCOMPACTION_ON_STARTUP_ENABLED("cassandra.autocompaction_on_startup_enabled", "true"), + /** When false, completely disables the auto-repair service including JMX registration */ + AUTOREPAIR_ENABLE("cassandra.autorepair.enable", "false"), AUTO_BOOTSTRAP("cassandra.auto_bootstrap"), AUTO_REPAIR_FREQUENCY_SECONDS("cassandra.auto_repair_frequency_seconds", convertToString(TimeUnit.MINUTES.toSeconds(5))), BATCHLOG_REPLAY_TIMEOUT_IN_MS("cassandra.batchlog.replay_timeout_in_ms"), diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 2c3d79183d49..8fc0ca4c6cea 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -41,6 +41,7 @@ import org.apache.cassandra.index.internal.CassandraIndex; import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.sstable.format.big.BigFormat; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.service.StartupChecks.StartupCheckType; import org.apache.cassandra.utils.StorageCompatibilityMode; @@ -348,6 +349,11 @@ public MemtableOptions() // The number of executors to use for building secondary indexes public volatile int concurrent_index_builders = 2; + // at least 20% of disk must be unused to run repair + // if you want to disable this feature (the recommendation is not to, but if you want to disable it for whatever reason) then set the ratio to 0.0 + @Replaces(oldName = "incremental_repair_disk_headroom_reject_ratio") + public volatile double repair_disk_headroom_reject_ratio = 0.2; + /** * @deprecated retry support removed on CASSANDRA-10992 */ @@ -592,6 +598,10 @@ public static class SSTableConfig @Replaces(oldName = "enable_materialized_views", converter = Converters.IDENTITY, deprecated = true) public boolean materialized_views_enabled = false; + // When true, materialized views data in SSTable go through commit logs during internodes streaming, e.g. repair + // When false, it behaves the same as normal streaming. + public volatile boolean materialized_views_on_repair_enabled = true; + @Replaces(oldName = "enable_transient_replication", converter = Converters.IDENTITY, deprecated = true) public boolean transient_replication_enabled = false; @@ -949,6 +959,8 @@ public static void setClientMode(boolean clientMode) public volatile DurationSpec.LongMicrosecondsBound minimum_timestamp_warn_threshold = null; public volatile DurationSpec.LongMicrosecondsBound minimum_timestamp_fail_threshold = null; + public volatile AutoRepairConfig auto_repair = new AutoRepairConfig(); + /** * The variants of paxos implementation and semantics supported by Cassandra. */ diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 68be14cd8f70..8f7057865af9 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -58,6 +58,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.common.util.concurrent.RateLimiter; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.utils.Pair; import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; @@ -4223,6 +4224,16 @@ public static void setMaterializedViewsEnabled(boolean enableMaterializedViews) conf.materialized_views_enabled = enableMaterializedViews; } + public static boolean isMaterializedViewsOnRepairEnabled() + { + return conf.materialized_views_on_repair_enabled; + } + + public static void setMaterializedViewsOnRepairEnabled(boolean val) + { + conf.materialized_views_on_repair_enabled = val; + } + public static boolean getSASIIndexesEnabled() { return conf.sasi_indexes_enabled; @@ -5309,4 +5320,29 @@ public static void setPaxosRepairRaceWait(boolean paxosRepairRaceWait) { conf.paxos_repair_race_wait = paxosRepairRaceWait; } + + public static AutoRepairConfig getAutoRepairConfig() + { + return conf.auto_repair; + } + + public static double getRepairDiskHeadroomRejectRatio() + { + return conf.repair_disk_headroom_reject_ratio; + } + + public static void setRepairDiskHeadroomRejectRatio(double value) + { + if (value < 0.0 || value > 1.0) + { + throw new IllegalArgumentException("Value must be >= 0 and <= 1 for repair_disk_headroom_reject_ratio"); + } + conf.repair_disk_headroom_reject_ratio = value; + } + + @VisibleForTesting + public static void setInitialTokens(String initial_token) + { + conf.initial_token = initial_token; + } } diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java index a0201c500a39..0c6802c84ebe 100644 --- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java +++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java @@ -449,6 +449,14 @@ public long getLong(String column) return LongType.instance.compose(data.get(column)); } + // this function will return the default value if the row doesn't have that column or the column data is null + // This function is used to avoid the nullpointerexception + public long getLong(String column, long ifNull) + { + ByteBuffer bytes = data.get(column); + return bytes == null ? ifNull : LongType.instance.compose(bytes); + } + public Set getSet(String column, AbstractType type) { ByteBuffer raw = data.get(column); diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index a538348ceab5..f021d4b8f358 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@ -82,7 +82,7 @@ public abstract class ModificationStatement implements CQLStatement.SingleKeyspa public static final String CUSTOM_EXPRESSIONS_NOT_ALLOWED = "Custom index expressions cannot be used in WHERE clauses for UPDATE or DELETE statements"; - private static final ColumnIdentifier CAS_RESULT_COLUMN = new ColumnIdentifier("[applied]", false); + public static final ColumnIdentifier CAS_RESULT_COLUMN = new ColumnIdentifier("[applied]", false); protected final StatementType type; diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index 93d477c8470d..40768b9ee093 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -25,6 +25,7 @@ import org.apache.cassandra.cql3.statements.PropertyDefinitions; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.schema.AutoRepairParams; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.CompressionParams; @@ -151,6 +152,9 @@ private TableParams build(TableParams.Builder builder) if (hasOption(READ_REPAIR)) builder.readRepair(ReadRepairStrategy.fromString(getString(READ_REPAIR))); + if (hasOption(Option.AUTO_REPAIR)) + builder.automatedRepair(AutoRepairParams.fromMap(getMap(Option.AUTO_REPAIR))); + return builder.build(); } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java index 9b38336a04b3..3ebc88ee9611 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java @@ -469,7 +469,6 @@ public void setCDCBlockWrites(boolean val) logger.info("Updated CDC block_writes from {} to {}", oldVal, val); } - @Override public boolean isCDCOnRepairEnabled() { diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java index 16079ff8883d..348b9a413466 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java @@ -865,7 +865,7 @@ private static void mutateFullyContainedSSTables(ColumnFamilyStore cfs, Set fullyContainedSSTables = findSSTablesToAnticompact(sstableIterator, normalizedRanges, sessionID); - cfs.metric.bytesMutatedAnticompaction.inc(SSTableReader.getTotalBytes(fullyContainedSSTables)); + cfs.metric.bytesMutatedAnticompaction.mark(SSTableReader.getTotalBytes(fullyContainedSSTables)); cfs.getCompactionStrategyManager().mutateRepaired(fullyContainedSSTables, UNREPAIRED_SSTABLE, sessionID, isTransient); // since we're just re-writing the sstable metdata for the fully contained sstables, we don't want // them obsoleted when the anti-compaction is complete. So they're removed from the transaction here @@ -1700,7 +1700,7 @@ private void doAntiCompaction(ColumnFamilyStore cfs, // repairedAt values for these, we still avoid anti-compacting already repaired sstables, as we currently don't // make use of any actual repairedAt value and splitting up sstables just for that is not worth it at this point. Set unrepairedSSTables = sstables.stream().filter((s) -> !s.isRepaired()).collect(Collectors.toSet()); - cfs.metric.bytesAnticompacted.inc(SSTableReader.getTotalBytes(unrepairedSSTables)); + cfs.metric.bytesAnticompacted.mark(SSTableReader.getTotalBytes(unrepairedSSTables)); Collection> groupedSSTables = cfs.getCompactionStrategyManager().groupSSTablesForAntiCompaction(unrepairedSSTables); // iterate over sstables to check if the full / transient / unrepaired ranges intersect them. diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index 50f87c799ece..d248ceae542b 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Set; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import org.slf4j.Logger; @@ -175,7 +176,7 @@ private boolean hasCDC(ColumnFamilyStore cfs) return cfs.metadata().params.cdc; } - // returns true iif it is a cdc table and cdc on repair is enabled. + // returns true if it is a cdc table and cdc on repair is enabled. private boolean cdcRequiresWriteCommitLog(ColumnFamilyStore cfs) { return DatabaseDescriptor.isCDCOnRepairEnabled() && hasCDC(cfs); @@ -190,11 +191,12 @@ private boolean cdcRequiresWriteCommitLog(ColumnFamilyStore cfs) * For CDC-enabled tables and write path for CDC is enabled, we want to ensure that the mutations are * run through the CommitLog, so they can be archived by the CDC process on discard. */ - private boolean requiresWritePath(ColumnFamilyStore cfs) + @VisibleForTesting + boolean requiresWritePath(ColumnFamilyStore cfs) { return cdcRequiresWriteCommitLog(cfs) || cfs.streamToMemtable() - || (session.streamOperation().requiresViewBuild() && hasViews(cfs)); + || (session.streamOperation().requiresViewBuild() && hasViews(cfs) && DatabaseDescriptor.isMaterializedViewsOnRepairEnabled()); } private void sendThroughWritePath(ColumnFamilyStore cfs, Collection readers) diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index 3d4bc95cacfd..379ef249949a 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -163,7 +163,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean, /* live member set */ @VisibleForTesting - final Set liveEndpoints = new ConcurrentSkipListSet<>(); + public final Set liveEndpoints = new ConcurrentSkipListSet<>(); /* unreachable member set */ private final Map unreachableEndpoints = new ConcurrentHashMap<>(); diff --git a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java index c954a05b1ce4..be86d5fd5692 100644 --- a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java +++ b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java @@ -25,15 +25,22 @@ import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.nio.ByteBuffer; -import java.util.regex.Pattern; +import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.base.Splitter; import com.google.common.net.HostAndPort; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; @@ -58,6 +65,7 @@ public final class InetAddressAndPort extends InetSocketAddress implements Comparable, Serializable { private static final long serialVersionUID = 0; + private static final Logger logger = LoggerFactory.getLogger(InetAddressAndPort.class); //Store these here to avoid requiring DatabaseDescriptor to be loaded. DatabaseDescriptor will set //these when it loads the config. A lot of unit tests won't end up loading DatabaseDescriptor. @@ -311,6 +319,52 @@ public static void initializeDefaultPort(int port) defaultPort = port; } + public static List stringify(Iterable endpoints) + { + return stringify(endpoints, true); + } + + public static List stringify(Iterable endpoints, boolean withPort) + { + List stringEndpoints = new ArrayList<>(); + for (InetAddressAndPort ep : endpoints) + { + stringEndpoints.add(ep.getHostAddress(withPort)); + } + return stringEndpoints; + } + + /** + * Parses a comma-separated list of hosts to a set of {@link InetAddressAndPort} + * + * @param value the comma-separated list of hosts to parse + * @param failOnError whether to fail when encountering an invalid hostname + * @return the set of parsed {@link InetAddressAndPort} + */ + public static Set parseHosts(String value, boolean failOnError) + { + Set hosts = new HashSet<>(); + for (String host : Splitter.on(',').split(value)) + { + try + { + hosts.add(InetAddressAndPort.getByName(host)); + } + catch (UnknownHostException e) + { + if (failOnError) + { + throw new IllegalArgumentException("Failed to parse host: " + host, e); + } + else + { + logger.warn("Invalid ip address {} from input={}", host, value); + } + } + } + return hosts; + } + static int getDefaultPort() { return defaultPort; diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java new file mode 100644 index 000000000000..e1355dbfa6cd --- /dev/null +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java @@ -0,0 +1,254 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.metrics; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; +import com.google.common.annotations.VisibleForTesting; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.autorepair.AutoRepair; +import org.apache.cassandra.service.AutoRepairService; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +/** + * Metrics related to AutoRepair. + */ +public class AutoRepairMetrics +{ + public final Gauge repairsInProgress; + public final Gauge nodeRepairTimeInSec; + public final Gauge clusterRepairTimeInSec; + public final Gauge longestUnrepairedSec; + public final Gauge repairStartLagSec; + public final Gauge succeededTokenRangesCount; + public final Gauge failedTokenRangesCount; + public final Gauge skippedTokenRangesCount; + public final Gauge skippedTablesCount; + public final Gauge totalMVTablesConsideredForRepair; + public final Gauge totalDisabledRepairTables; + public final Gauge totalBytesToRepair; + public final Gauge bytesAlreadyRepaired; + public final Gauge totalKeyspaceRepairPlansToRepair; + public final Gauge keyspaceRepairPlansAlreadyRepaired; + + public Counter repairTurnMyTurn; + public Counter repairTurnMyTurnDueToPriority; + public Counter repairTurnMyTurnForceRepair; + public Counter repairDelayedByReplica; + public Counter repairDelayedBySchedule; + + private final RepairType repairType; + + private volatile int repairStartLagSecVal; + + public AutoRepairMetrics(RepairType repairType) + { + this.repairType = repairType; + AutoRepairMetricsFactory factory = new AutoRepairMetricsFactory(repairType); + + repairsInProgress = Metrics.register(factory.createMetricName("RepairsInProgress"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).isRepairInProgress() ? 1 : 0; + } + }); + + nodeRepairTimeInSec = Metrics.register(factory.createMetricName("NodeRepairTimeInSec"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getNodeRepairTimeInSec(); + } + }); + + clusterRepairTimeInSec = Metrics.register(factory.createMetricName("ClusterRepairTimeInSec"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getClusterRepairTimeInSec(); + } + }); + + skippedTokenRangesCount = Metrics.register(factory.createMetricName("SkippedTokenRangesCount"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getSkippedTokenRangesCount(); + } + }); + + skippedTablesCount = Metrics.register(factory.createMetricName("SkippedTablesCount"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getSkippedTablesCount(); + } + }); + + longestUnrepairedSec = Metrics.register(factory.createMetricName("LongestUnrepairedSec"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getLongestUnrepairedSec(); + } + }); + + repairStartLagSec = Metrics.register(factory.createMetricName("RepairStartLagSec"), new Gauge() + { + public Integer getValue() + { + return repairStartLagSecVal; + } + }); + + succeededTokenRangesCount = Metrics.register(factory.createMetricName("SucceededTokenRangesCount"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getSucceededTokenRangesCount(); + } + }); + + failedTokenRangesCount = Metrics.register(factory.createMetricName("FailedTokenRangesCount"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getFailedTokenRangesCount(); + } + }); + + repairTurnMyTurn = Metrics.counter(factory.createMetricName("RepairTurnMyTurn")); + repairTurnMyTurnDueToPriority = Metrics.counter(factory.createMetricName("RepairTurnMyTurnDueToPriority")); + repairTurnMyTurnForceRepair = Metrics.counter(factory.createMetricName("RepairTurnMyTurnForceRepair")); + + repairDelayedByReplica = Metrics.counter(factory.createMetricName("RepairDelayedByReplica")); + repairDelayedBySchedule = Metrics.counter(factory.createMetricName("RepairDelayedBySchedule")); + + totalMVTablesConsideredForRepair = Metrics.register(factory.createMetricName("TotalMVTablesConsideredForRepair"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getTotalMVTablesConsideredForRepair(); + } + }); + + totalDisabledRepairTables = Metrics.register(factory.createMetricName("TotalDisabledRepairTables"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getTotalDisabledTablesRepairCount(); + } + }); + totalBytesToRepair = Metrics.register(factory.createMetricName("TotalBytesToRepair"), new Gauge() + { + public Long getValue() + { + return AutoRepair.instance.getRepairState(repairType).getTotalBytesToRepair(); + } + }); + bytesAlreadyRepaired = Metrics.register(factory.createMetricName("BytesAlreadyRepaired"), new Gauge() + { + public Long getValue() + { + return AutoRepair.instance.getRepairState(repairType).getBytesAlreadyRepaired(); + } + }); + totalKeyspaceRepairPlansToRepair = Metrics.register(factory.createMetricName("TotalKeyspaceRepairPlansToRepair"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getTotalKeyspaceRepairPlansToRepair(); + } + }); + keyspaceRepairPlansAlreadyRepaired = Metrics.register(factory.createMetricName("KeyspaceRepairPlansAlreadyRepaired"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getKeyspaceRepairPlansAlreadyRepaired(); + } + }); + } + + public void recordTurn(AutoRepairUtils.RepairTurn turn) + { + switch (turn) + { + case MY_TURN: + repairTurnMyTurn.inc(); + break; + case MY_TURN_FORCE_REPAIR: + repairTurnMyTurnForceRepair.inc(); + break; + case MY_TURN_DUE_TO_PRIORITY: + repairTurnMyTurnDueToPriority.inc(); + break; + default: + throw new RuntimeException(String.format("Unrecoginized turn: %s", turn.name())); + } + this.repairStartLagSecVal = 0; + } + + /** + * Record perceived lag in scheduling repair. + *

+ * Takes the current time and subtracts it from the given last repair finish time. It then compares the difference + * with the min repair interval for this repair type, and if that value is greater than 0, records it. + */ + public void recordRepairStartLag(long lastFinishTimeInMs) + { + long now = AutoRepair.instance.currentTimeMs(); + long deltaFinish = now - lastFinishTimeInMs; + long deltaMinRepairInterval = deltaFinish - AutoRepairService.instance + .getAutoRepairConfig().getRepairMinInterval(repairType) + .toMilliseconds(); + this.repairStartLagSecVal = deltaMinRepairInterval > 0 ? (int) MILLISECONDS.toSeconds(deltaMinRepairInterval) : 0; + } + + @VisibleForTesting + protected static class AutoRepairMetricsFactory implements MetricNameFactory + { + private static final String TYPE = "AutoRepair"; + @VisibleForTesting + protected final String repairType; + + protected AutoRepairMetricsFactory(RepairType repairType) + { + this.repairType = repairType.toString().toLowerCase(); + } + + @Override + public CassandraMetricsRegistry.MetricName createMetricName(String metricName) + { + StringBuilder mbeanName = new StringBuilder(); + mbeanName.append(DefaultNameFactory.GROUP_NAME).append(':'); + mbeanName.append("type=").append(TYPE); + mbeanName.append(",name=").append(metricName); + mbeanName.append(",repairType=").append(repairType); + + StringBuilder scope = new StringBuilder(); + scope.append("repairType=").append(repairType); + + return new CassandraMetricsRegistry.MetricName(DefaultNameFactory.GROUP_NAME, TYPE.toLowerCase(), + metricName, scope.toString(), mbeanName.toString()); + } + } +} diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java new file mode 100644 index 000000000000..e97ce34e5a73 --- /dev/null +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.metrics; + +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * AutoRepair metrics manager holding all the auto-repair related metrics. + */ +public class AutoRepairMetricsManager +{ + private static final Map metrics = new ConcurrentHashMap<>(); + + public static AutoRepairMetrics getMetrics(RepairType repairType) + { + return metrics.computeIfAbsent(repairType, k -> new AutoRepairMetrics(repairType)); + } +} diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java index 598e484f6bdf..d104fecb502b 100644 --- a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java +++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java @@ -68,20 +68,30 @@ public Counter counter(MetricName name, MetricName alias) } public Meter meter(MetricName name) + { + return meter(name, false); + } + + public Meter meter(MetricName name, boolean gaugeCompatible) { Meter meter = meter(name.getMetricName()); - registerMBean(meter, name.getMBeanName()); + registerMBean(meter, name.getMBeanName(), gaugeCompatible); return meter; } - public Meter meter(MetricName name, MetricName alias) + public Meter meter(MetricName name, MetricName alias, boolean gaugeCompatible) { - Meter meter = meter(name); - registerAlias(name, alias); + Meter meter = meter(name, gaugeCompatible); + registerAlias(name, alias, gaugeCompatible); return meter; } + public Meter meter(MetricName name, MetricName alias) + { + return meter(name, alias, false); + } + public Histogram histogram(MetricName name, boolean considerZeroes) { Histogram histogram = register(name, new ClearableHistogram(new DecayingEstimatedHistogramReservoir(considerZeroes))); @@ -217,6 +227,11 @@ public boolean remove(MetricName name, MetricName... aliases) } public void registerMBean(Metric metric, ObjectName name) + { + registerMBean(metric, name, false); + } + + public void registerMBean(Metric metric, ObjectName name, boolean gaugeCompatible) { AbstractBean mbean; @@ -229,7 +244,18 @@ else if (metric instanceof Histogram) else if (metric instanceof Timer) mbean = new JmxTimer((Timer) metric, name, TimeUnit.SECONDS, DEFAULT_TIMER_UNIT); else if (metric instanceof Metered) - mbean = new JmxMeter((Metered) metric, name, TimeUnit.SECONDS); + { + // If a gauge compatible meter is requested, create a special implementation which + // also yields a 'Value' attribute for backwards compatibility. + if (gaugeCompatible) + { + mbean = new JmxMeterGaugeCompatible((Metered) metric, name, TimeUnit.SECONDS); + } + else + { + mbean = new JmxMeter((Metered) metric, name, TimeUnit.SECONDS); + } + } else throw new IllegalArgumentException("Unknown metric type: " + metric.getClass()); @@ -238,11 +264,16 @@ else if (metric instanceof Metered) } private void registerAlias(MetricName existingName, MetricName aliasName) + { + registerAlias(existingName, aliasName, false); + } + + private void registerAlias(MetricName existingName, MetricName aliasName, boolean gaugeCompatible) { Metric existing = Metrics.getMetrics().get(existingName.getMetricName()); assert existing != null : existingName + " not registered"; - registerMBean(existing, aliasName.getMBeanName()); + registerMBean(existing, aliasName.getMBeanName(), gaugeCompatible); } private void removeAlias(MetricName name) @@ -531,6 +562,30 @@ private String calculateRateUnit(TimeUnit unit) } } + public interface JmxMeterGaugeCompatibleMBean extends JmxMeterMBean, JmxGaugeMBean {} + + /** + * An implementation of {@link JmxMeter} that is compatible with {@link JmxGaugeMBean} in that it also + * implements {@link JmxGaugeMBean}. This is useful for metrics that were migrated from {@link JmxGauge} + * to {@link JmxMeter} like {@link TableMetrics#bytesAnticompacted} and + * {@link TableMetrics#bytesMutatedAnticompaction}. + */ + private static class JmxMeterGaugeCompatible extends JmxMeter implements JmxMeterGaugeCompatibleMBean + { + + private JmxMeterGaugeCompatible(Metered metric, ObjectName objectName, TimeUnit rateUnit) + { + super(metric, objectName, rateUnit); + } + + @Override + public Object getValue() + { + return getCount(); + } + } + + public interface JmxTimerMBean extends JmxMeterMBean { double getMin(); diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java index ae15bbf95a04..5ddab582c06a 100644 --- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java +++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java @@ -184,6 +184,12 @@ public class KeyspaceMetrics public final ImmutableMap, ImmutableMap>> formatSpecificGauges; + public final Meter bytesAnticompacted; + public final Meter bytesMutatedAnticompaction; + public final Meter bytesPreviewed; + public final Meter tokenRangesPreviewedDesynchronized; + public final Meter bytesPreviewedDesynchronized; + public final MetricNameFactory factory; private final Keyspace keyspace; @@ -292,6 +298,11 @@ public KeyspaceMetrics(final Keyspace ks) outOfRangeTokenReads = createKeyspaceCounter("ReadOutOfRangeToken"); outOfRangeTokenWrites = createKeyspaceCounter("WriteOutOfRangeToken"); outOfRangeTokenPaxosRequests = createKeyspaceCounter("PaxosOutOfRangeToken"); + bytesAnticompacted = createKeyspaceMeter("BytesAnticompacted"); + bytesMutatedAnticompaction = createKeyspaceMeter("BytesMutatedAnticompaction"); + bytesPreviewed = createKeyspaceMeter("BytesPreviewed"); + tokenRangesPreviewedDesynchronized = createKeyspaceMeter("TokenRangesPreviewedDesynchronized"); + bytesPreviewedDesynchronized = createKeyspaceMeter("BytesPreviewedDesynchronized"); } /** diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index b2a8e61fe4ca..49f132744618 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -202,9 +202,15 @@ public class TableMetrics /** number of partitions read creating merkle trees */ public final TableHistogram partitionsValidated; /** number of bytes read while doing anticompaction */ - public final Counter bytesAnticompacted; + public final TableMeter bytesAnticompacted; /** number of bytes where the whole sstable was contained in a repairing range so that we only mutated the repair status */ - public final Counter bytesMutatedAnticompaction; + public final TableMeter bytesMutatedAnticompaction; + /** number of bytes that were scanned during preview repair */ + public final TableMeter bytesPreviewed; + /** number of desynchronized token ranges that were detected during preview repair */ + public final TableMeter tokenRangesPreviewedDesynchronized; + /** number of desynchronized bytes that were detected during preview repair */ + public final TableMeter bytesPreviewedDesynchronized; /** ratio of how much we anticompact vs how much we could mutate the repair status*/ public final Gauge mutatedAnticompactionGauge; @@ -811,12 +817,15 @@ public Long getValue() bytesValidated = createTableHistogram("BytesValidated", cfs.keyspace.metric.bytesValidated, false); partitionsValidated = createTableHistogram("PartitionsValidated", cfs.keyspace.metric.partitionsValidated, false); - bytesAnticompacted = createTableCounter("BytesAnticompacted"); - bytesMutatedAnticompaction = createTableCounter("BytesMutatedAnticompaction"); + bytesAnticompacted = createTableMeter("BytesAnticompacted", cfs.keyspace.metric.bytesAnticompacted, true); + bytesMutatedAnticompaction = createTableMeter("BytesMutatedAnticompaction", cfs.keyspace.metric.bytesMutatedAnticompaction, true); + bytesPreviewed = createTableMeter("BytesPreviewed", cfs.keyspace.metric.bytesPreviewed); + tokenRangesPreviewedDesynchronized = createTableMeter("TokenRangesPreviewedDesynchronized", cfs.keyspace.metric.tokenRangesPreviewedDesynchronized); + bytesPreviewedDesynchronized = createTableMeter("BytesPreviewedDesynchronized", cfs.keyspace.metric.bytesPreviewedDesynchronized); mutatedAnticompactionGauge = createTableGauge("MutatedAnticompactionGauge", () -> { - double bytesMutated = bytesMutatedAnticompaction.getCount(); - double bytesAnticomp = bytesAnticompacted.getCount(); + double bytesMutated = bytesMutatedAnticompaction.table.getCount(); + double bytesAnticomp = bytesAnticompacted.table.getCount(); if (bytesAnticomp + bytesMutated > 0) return bytesMutated / (bytesAnticomp + bytesMutated); return 0.0; @@ -1094,17 +1103,23 @@ protected SnapshottingTimer createTableTimer(String name) protected TableMeter createTableMeter(String name, Meter keyspaceMeter) { - return createTableMeter(name, name, keyspaceMeter); + return createTableMeter(name, keyspaceMeter, false); } - protected TableMeter createTableMeter(String name, String alias, Meter keyspaceMeter) + protected TableMeter createTableMeter(String name, Meter keyspaceMeter, boolean globalMeterGaugeCompatible) + { + return createTableMeter(name, name, keyspaceMeter, globalMeterGaugeCompatible); + } + + protected TableMeter createTableMeter(String name, String alias, Meter keyspaceMeter, boolean globalMeterGaugeCompatible) { Meter meter = Metrics.meter(factory.createMetricName(name), aliasFactory.createMetricName(alias)); register(name, alias, meter); return new TableMeter(meter, keyspaceMeter, Metrics.meter(GLOBAL_FACTORY.createMetricName(name), - GLOBAL_ALIAS_FACTORY.createMetricName(alias))); + GLOBAL_ALIAS_FACTORY.createMetricName(alias), + globalMeterGaugeCompatible)); } private LatencyMetrics createLatencyMetrics(String namePrefix, LatencyMetrics ... parents) @@ -1176,10 +1191,15 @@ private TableMeter(Meter table, Meter keyspace, Meter global) } public void mark() + { + mark(1L); + } + + public void mark(long val) { for (Meter meter : all) { - meter.mark(); + meter.mark(val); } } } diff --git a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java index edee11cf2007..600e64610add 100644 --- a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java +++ b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java @@ -26,6 +26,7 @@ import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; @@ -85,10 +86,10 @@ public Future performUnsafe(ExecutorPlus executor, Sche else { message = (previewKind == PreviewKind.REPAIRED ? "Repaired data is inconsistent\n" : "Preview complete\n") + summary; - RepairMetrics.previewFailures.inc(); if (previewKind == PreviewKind.REPAIRED) maybeSnapshotReplicas(parentSession, keyspace, result.results.get()); // we know its present as summary used it } + emitMetrics(summary); successMessage += "; " + message; coordinator.notification(message); @@ -96,6 +97,21 @@ public Future performUnsafe(ExecutorPlus executor, Sche }); } + private void emitMetrics(SyncStatSummary summary) + { + if (!summary.isEmpty()) + RepairMetrics.previewFailures.inc(); + + summary.getTotals().forEach((key, table) -> { + if (table.isCounter()) + return; + + ColumnFamilyStore cfs = Keyspace.open(key.left).getColumnFamilyStore(key.right); + cfs.metric.tokenRangesPreviewedDesynchronized.mark(table.getRanges()); + cfs.metric.bytesPreviewedDesynchronized.mark(table.getBytes()); + }); + } + private void maybeSnapshotReplicas(TimeUUID parentSession, String keyspace, List results) { if (!DatabaseDescriptor.snapshotOnRepairedDataMismatch()) diff --git a/src/java/org/apache/cassandra/repair/RepairCoordinator.java b/src/java/org/apache/cassandra/repair/RepairCoordinator.java index 091819c65d7b..b5ad02cd216a 100644 --- a/src/java/org/apache/cassandra/repair/RepairCoordinator.java +++ b/src/java/org/apache/cassandra/repair/RepairCoordinator.java @@ -32,6 +32,7 @@ import java.util.function.Function; import java.util.function.Supplier; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -102,7 +103,8 @@ public class RepairCoordinator implements Runnable, ProgressEventNotifier, Repai private final Function getLocalReplicas; private final List listeners = new ArrayList<>(); - private final AtomicReference firstError = new AtomicReference<>(null); + @VisibleForTesting + protected final AtomicReference firstError = new AtomicReference<>(null); final SharedContext ctx; final Scheduler validationScheduler; @@ -208,12 +210,12 @@ private void success(String msg) complete(null); } - private void fail(String reason) + protected void fail(String reason) { if (reason == null) { Throwable error = firstError.get(); - reason = error != null ? error.toString() : "Some repair failed"; + reason = (error != null && error.getMessage() != null) ? error.getMessage() : "Some repair failed"; } state.phase.fail(reason); ParticipateState p = ctx.repair().participate(state.id); diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java index 621aacc378da..1f4fa5b5683e 100644 --- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java +++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java @@ -114,6 +114,13 @@ public void doVerb(final Message message) replyDedup(ctx.repair().participate(state.id), message); return; } + if (!ctx.repair().verifyDiskHeadroomThreshold(prepareMessage.parentRepairSession, prepareMessage.previewKind)) + { + // error is logged in verifyDiskHeadroomThreshold + state.phase.fail("Not enough disk headroom to perform repair"); + sendFailureResponse(message); + return; + } if (!ctx.repair().verifyCompactionsPendingThreshold(prepareMessage.parentRepairSession, prepareMessage.previewKind)) { // error is logged in verifyCompactionsPendingThreshold diff --git a/src/java/org/apache/cassandra/repair/ValidationManager.java b/src/java/org/apache/cassandra/repair/ValidationManager.java index e3598cd38f87..c4f4d7419c04 100644 --- a/src/java/org/apache/cassandra/repair/ValidationManager.java +++ b/src/java/org/apache/cassandra/repair/ValidationManager.java @@ -37,6 +37,7 @@ import org.apache.cassandra.metrics.TableMetrics; import org.apache.cassandra.metrics.TopPartitionTracker; import org.apache.cassandra.repair.state.ValidationState; +import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MerkleTree; @@ -143,6 +144,10 @@ public static void doValidation(ColumnFamilyStore cfs, Validator validator) thro { cfs.metric.bytesValidated.update(state.estimatedTotalBytes); cfs.metric.partitionsValidated.update(state.partitionsProcessed); + if (validator.getPreviewKind() != PreviewKind.NONE) + { + cfs.metric.bytesPreviewed.mark(state.estimatedTotalBytes); + } if (topPartitionCollector != null) cfs.topPartitions.merge(topPartitionCollector); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java new file mode 100644 index 000000000000..692ef6641efa --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -0,0 +1,629 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.EnumMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Supplier; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.Uninterruptibles; + +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.utils.Clock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.concurrent.ScheduledExecutorPlus; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn; +import org.apache.cassandra.utils.concurrent.Condition; +import org.apache.cassandra.utils.concurrent.Future; +import org.apache.cassandra.utils.progress.ProgressEvent; +import org.apache.cassandra.utils.progress.ProgressEventType; +import org.apache.cassandra.utils.progress.ProgressListener; + +import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; +import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition; + +/** + * AutoRepair scheduler responsible for running different types of repairs. + */ +public class AutoRepair +{ + private static final Logger logger = LoggerFactory.getLogger(AutoRepair.class); + private static final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS"); + + @VisibleForTesting + protected static Supplier timeFunc = Clock.Global::currentTimeMillis; + + // Sleep for 5 seconds if repair finishes quickly to flush JMX metrics; it happens only for Cassandra nodes with tiny amount of data. + public static DurationSpec.IntSecondsBound SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("5s"); + + @VisibleForTesting + public Map repairStates; + + @VisibleForTesting + protected Map repairExecutors; + + protected Map repairRunnableExecutors; + + @VisibleForTesting + // Auto-repair is likely to be run on multiple nodes independently, we want to avoid running multiple repair + // sessions on overlapping datasets at the same time. Shuffling keyspaces reduces the likelihood of this happening. + protected static Consumer> shuffleFunc = java.util.Collections::shuffle; + + @VisibleForTesting + protected static BiConsumer sleepFunc = Uninterruptibles::sleepUninterruptibly; + + @VisibleForTesting + public boolean isSetupDone = false; + public static AutoRepair instance = new AutoRepair(); + + public volatile boolean isShutDown = false; + + private AutoRepair() + { + // Private constructor to prevent instantiation + } + + public void setup() + { + // Ensure setup is done only once; this is only for unit tests + // For production, this method should be called only once. + synchronized (this) + { + if (isSetupDone) + { + return; + } + repairExecutors = new EnumMap<>(AutoRepairConfig.RepairType.class); + repairRunnableExecutors = new EnumMap<>(AutoRepairConfig.RepairType.class); + repairStates = new EnumMap<>(AutoRepairConfig.RepairType.class); + AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); + + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + repairExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-Repair-" + repairType.getConfigName(), Thread.NORM_PRIORITY)); + repairRunnableExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-RepairRunnable-" + repairType.getConfigName(), Thread.NORM_PRIORITY)); + repairStates.put(repairType, AutoRepairConfig.RepairType.getAutoRepairState(repairType, config)); + } + + AutoRepairUtils.setup(); + + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + if (config.isAutoRepairEnabled(repairType)) + AutoRepairService.instance.checkCanRun(repairType); + + repairExecutors.get(repairType).scheduleWithFixedDelay( + () -> repair(repairType), + config.getInitialSchedulerDelay(repairType).toSeconds(), + config.getRepairCheckInterval().toSeconds(), + TimeUnit.SECONDS); + } + isSetupDone = true; + } + } + + /** + * @return The current observed system time in ms. + */ + public long currentTimeMs() + { + return timeFunc.get(); + } + + // repair runs a repair session of the given type synchronously. + public void repair(AutoRepairConfig.RepairType repairType) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + if (!config.isAutoRepairEnabled(repairType)) + { + logger.debug("Auto-repair is disabled for repair type {}", repairType); + return; + } + if (!config.isMixedMajorVersionRepairEnabled() && AutoRepairUtils.hasMultipleLiveMajorVersions()) + { + logger.info("Auto-repair is disabled when nodes in the cluster have different major versions"); + return; + } + if (AutoRepairUtils.hasNodesBelowMinimumVersion()) + { + logger.info("Auto-repair is disabled because some nodes are running unsupported versions " + + "(auto-repair requires all nodes to be above version {})", + AutoRepairUtils.LAST_UNSUPPORTED_VERSION_FOR_AUTO_REPAIR); + return; + } + AutoRepairService.instance.checkCanRun(repairType); + AutoRepairState repairState = repairStates.get(repairType); + try + { + String localDC = DatabaseDescriptor.getLocalDataCenter(); + if (config.getIgnoreDCs(repairType).contains(localDC)) + { + logger.info("Not running repair as this node belongs to datacenter {}", localDC); + return; + } + + // refresh the longest unrepaired node + repairState.setLongestUnrepairedNode(AutoRepairUtils.getHostWithLongestUnrepairTime(repairType)); + + //consistency level to use for local query + UUID myId = Gossiper.instance.getHostId(FBUtilities.getBroadcastAddressAndPort()); + + // If it's too soon to run repair, don't bother checking if it's our turn. + if (tooSoonToRunRepair(repairType, repairState, config, myId)) + { + return; + } + + RepairTurn turn = AutoRepairUtils.myTurnToRunRepair(repairType, myId); + if (turn == MY_TURN || turn == MY_TURN_DUE_TO_PRIORITY || turn == MY_TURN_FORCE_REPAIR) + { + repairState.recordTurn(turn); + repairState.setBytesAlreadyRepaired(0L); + repairState.setKeyspaceRepairPlansAlreadyRepaired(0); + // For normal auto repair, we will use primary range only repairs (Repair with -pr option). + // For some cases, we may set the auto_repair_primary_token_range_only flag to false then we will do repair + // without -pr. We may also do force repair for certain node that we want to repair all the data on one node + // When doing force repair, we want to repair without -pr. + boolean primaryRangeOnly = config.getRepairPrimaryTokenRangeOnly(repairType) + && turn != MY_TURN_FORCE_REPAIR; + + long startTimeInMillis = timeFunc.get(); + logger.info("My host id: {}, my turn to run repair...repair primary-ranges only? {}", myId, + config.getRepairPrimaryTokenRangeOnly(repairType)); + AutoRepairUtils.updateStartAutoRepairHistory(repairType, myId, timeFunc.get(), turn); + + repairState.setRepairKeyspaceCount(0); + repairState.setRepairInProgress(true); + repairState.setTotalTablesConsideredForRepair(0); + repairState.setTotalMVTablesConsideredForRepair(0); + + CollectedRepairStats collectedRepairStats = new CollectedRepairStats(); + + List keyspaces = new ArrayList<>(); + Keyspace.all().forEach(keyspaces::add); + // Filter out keyspaces and tables to repair and group into a map by keyspace. + Map> keyspacesAndTablesToRepair = new LinkedHashMap<>(); + for (Keyspace keyspace : keyspaces) + { + if (!AutoRepairUtils.shouldConsiderKeyspace(keyspace)) + { + continue; + } + List tablesToBeRepairedList = retrieveTablesToBeRepaired(keyspace, config, repairType, repairState, collectedRepairStats); + keyspacesAndTablesToRepair.put(keyspace.getName(), tablesToBeRepairedList); + } + + // Separate out the keyspaces and tables to repair based on their priority, with each repair plan representing a uniquely occuring priority. + List repairPlans = PrioritizedRepairPlan.build(keyspacesAndTablesToRepair, repairType, shuffleFunc, primaryRangeOnly); + repairState.updateRepairScheduleStatistics(repairPlans); + + // calculate the repair assignments for each priority:keyspace. + Iterator repairAssignmentsIterator = config.getTokenRangeSplitterInstance(repairType).getRepairAssignments(primaryRangeOnly, repairPlans); + + int keyspaceRepairAssignmentsAlreadyRepaired = 0; + while (repairAssignmentsIterator.hasNext()) + { + KeyspaceRepairAssignments repairAssignments = repairAssignmentsIterator.next(); + List assignments = repairAssignments.getRepairAssignments(); + if (assignments.isEmpty()) + { + keyspaceRepairAssignmentsAlreadyRepaired++; + logger.info("Skipping repairs for priorityBucket={} for keyspace={} since it yielded no assignments", repairAssignments.getPriority(), repairAssignments.getKeyspaceName()); + continue; + } + + logger.info("Submitting repairs for priorityBucket={} for keyspace={} with assignmentCount={} and keyspaceRepairAssignmentsAlreadyRepaired={}/{}", + repairAssignments.getPriority(), repairAssignments.getKeyspaceName(), repairAssignments.getRepairAssignments().size(), + keyspaceRepairAssignmentsAlreadyRepaired, repairState.getTotalKeyspaceRepairPlansToRepair()); + repairKeyspace(repairType, primaryRangeOnly, repairAssignments.getKeyspaceName(), repairAssignments.getRepairAssignments(), collectedRepairStats); + keyspaceRepairAssignmentsAlreadyRepaired++; + repairState.setKeyspaceRepairPlansAlreadyRepaired(keyspaceRepairAssignmentsAlreadyRepaired); + } + + cleanupAndUpdateStats(turn, repairType, repairState, myId, startTimeInMillis, collectedRepairStats); + } + else + { + logger.info("Waiting for my turn..."); + } + } + catch (Exception e) + { + logger.error("Exception in autorepair:", e); + } + } + + private void repairKeyspace(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List repairAssignments, CollectedRepairStats collectedRepairStats) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AutoRepairState repairState = repairStates.get(repairType); + + // evaluate over each keyspace's repair assignments. + repairState.setRepairKeyspaceCount(repairState.getRepairKeyspaceCount() + 1); + + int totalRepairAssignments = repairAssignments.size(); + long keyspaceStartTime = timeFunc.get(); + RepairAssignment previousAssignment = null; + long tableStartTime = timeFunc.get(); + int totalProcessedAssignments = 0; + Set> ranges = new HashSet<>(); + long bytesAlreadyRepaired = repairState.getBytesAlreadyRepaired(); + for (RepairAssignment curRepairAssignment : repairAssignments) + { + try + { + totalProcessedAssignments++; + boolean repairOneTableAtATime = !config.getRepairByKeyspace(repairType); + if (previousAssignment != null && repairOneTableAtATime && !previousAssignment.tableNames.equals(curRepairAssignment.tableNames)) + { + // In the repair assignment, all the tables are appended sequnetially. + // Check if we have a different table, and if so, we should reset the table start time. + tableStartTime = timeFunc.get(); + } + previousAssignment = curRepairAssignment; + if (!config.isAutoRepairEnabled(repairType)) + { + logger.error("Auto-repair for type {} is disabled hence not running repair", repairType); + repairState.setRepairInProgress(false); + return; + } + if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, repairAssignments.size())) + { + collectedRepairStats.skippedTokenRanges += totalRepairAssignments - totalProcessedAssignments; + logger.info("Keyspace took too much time to repair hence skipping it {}", + keyspaceName); + break; + } + if (repairOneTableAtATime && AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) + { + collectedRepairStats.skippedTokenRanges += 1; + logger.info("Table took too much time to repair hence skipping it table name {}.{}, token range {}", + keyspaceName, curRepairAssignment.tableNames, curRepairAssignment.tokenRange); + continue; + } + + Range tokenRange = curRepairAssignment.getTokenRange(); + logger.debug("Current Token Left side {}, right side {}", + tokenRange.left.toString(), + tokenRange.right.toString()); + + ranges.add(curRepairAssignment.getTokenRange()); + if ((totalProcessedAssignments % config.getRepairThreads(repairType) == 0) || + (totalProcessedAssignments == totalRepairAssignments)) + { + boolean success = false; + int retryCount = 0; + Future f = null; + while (retryCount <= config.getRepairMaxRetries(repairType)) + { + RepairCoordinator task = repairState.getRepairRunnable(keyspaceName, + Lists.newArrayList(curRepairAssignment.getTableNames()), + ranges, primaryRangeOnly); + RepairProgressListener listener = new RepairProgressListener(repairType); + task.addProgressListener(listener); + f = repairRunnableExecutors.get(repairType).submit(task); + try + { + long jobStartTime = timeFunc.get(); + listener.await(config.getRepairSessionTimeout(repairType)); + success = listener.isSuccess(); + soakAfterRepair(jobStartTime, config.getRepairTaskMinDuration().toMilliseconds()); + } + catch (InterruptedException e) + { + logger.error("Exception in cond await:", e); + } + if (success) + { + break; + } + else if (retryCount < config.getRepairMaxRetries(repairType)) + { + boolean cancellationStatus = f.cancel(true); + logger.warn("Repair failed for range {}-{} for {} tables {} with cancellationStatus: {} retrying after {} seconds...", + tokenRange.left, tokenRange.right, + keyspaceName, curRepairAssignment.getTableNames(), + cancellationStatus, config.getRepairRetryBackoff(repairType).toSeconds()); + sleepFunc.accept(config.getRepairRetryBackoff(repairType).toSeconds(), TimeUnit.SECONDS); + } + retryCount++; + } + //check repair status + if (success) + { + logger.info("Repair completed for range {}-{} for {} tables {}, total assignments: {}," + + "processed assignments: {}", tokenRange.left, tokenRange.right, + keyspaceName, curRepairAssignment.getTableNames(), totalRepairAssignments, totalProcessedAssignments); + collectedRepairStats.succeededTokenRanges += ranges.size(); + } + else + { + boolean cancellationStatus = true; + if (f != null) + { + cancellationStatus = f.cancel(true); + } + //in the future we can add retry, etc. + logger.error("Repair failed for range {}-{} for {} tables {} after {} retries, total assignments: {}," + + "processed assignments: {}, cancellationStatus: {}", tokenRange.left, tokenRange.right, keyspaceName, + curRepairAssignment.getTableNames(), retryCount, totalRepairAssignments, totalProcessedAssignments, cancellationStatus); + collectedRepairStats.failedTokenRanges += ranges.size(); + } + ranges.clear(); + } + bytesAlreadyRepaired += curRepairAssignment.getEstimatedBytes(); + repairState.setBytesAlreadyRepaired(bytesAlreadyRepaired); + logger.info("Repair completed for {} tables {}, range {}, bytesAlreadyRepaired {}/{}", + keyspaceName, curRepairAssignment.getTableNames(), curRepairAssignment.getTokenRange(), bytesAlreadyRepaired, repairState.getTotalBytesToRepair()); + } + catch (Exception e) + { + logger.error("Exception while repairing keyspace {}:", keyspaceName, e); + } + } + } + + private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoRepairState repairState, AutoRepairConfig config, UUID myId) + { + if (repairState.getLastRepairTime() == 0) + { + // the node has either just boooted or has not run repair before, + // we should check for the node's repair history in the DB + repairState.setLastRepairTime(AutoRepairUtils.getLastRepairTimeForNode(repairType, myId)); + } + /* + * check if it is too soon to run repair. one of the reason we + * should not run frequent repair is that repair triggers + * memtable flush + */ + long timeElapsedSinceLastRepair = TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - repairState.getLastRepairTime()); + if (timeElapsedSinceLastRepair < config.getRepairMinInterval(repairType).toSeconds()) + { + logger.info("Too soon to run repair, last repair was done {} seconds ago", + timeElapsedSinceLastRepair); + return true; + } + return false; + } + + private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairConfig config, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, CollectedRepairStats collectedRepairStats) + { + Tables tables = keyspace.getMetadata().tables; + List tablesToBeRepaired = new ArrayList<>(); + Iterator iter = tables.iterator(); + while (iter.hasNext()) + { + repairState.setTotalTablesConsideredForRepair(repairState.getTotalTablesConsideredForRepair() + 1); + TableMetadata tableMetadata = iter.next(); + String tableName = tableMetadata.name; + + ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); + if (!columnFamilyStore.metadata().params.autoRepair.repairEnabled(repairType)) + { + logger.info("Repair is disabled for keyspace {} for tables: {}", keyspace.getName(), tableName); + repairState.setTotalDisabledTablesRepairCount(repairState.getTotalDisabledTablesRepairCount() + 1); + collectedRepairStats.skippedTables++; + continue; + } + + // this is done to make autorepair safe as running repair on table with more sstables + // may have its own challenges + int totalSSTables = columnFamilyStore.getLiveSSTables().size(); + if (totalSSTables > config.getRepairSSTableCountHigherThreshold(repairType)) + { + logger.info("Too many SSTables for repair for table {}.{}" + + "totalSSTables {}", keyspace.getName(), tableName, totalSSTables); + collectedRepairStats.skippedTables++; + continue; + } + + tablesToBeRepaired.add(tableName); + + // See if we should repair MVs as well that are associated with this given table + List mvs = AutoRepairUtils.getAllMVs(repairType, keyspace, tableMetadata); + if (!mvs.isEmpty()) + { + tablesToBeRepaired.addAll(mvs); + repairState.setTotalMVTablesConsideredForRepair(repairState.getTotalMVTablesConsideredForRepair() + mvs.size()); + } + } + return tablesToBeRepaired; + } + + private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, UUID myId, + long startTimeInMillis, CollectedRepairStats collectedRepairStats) throws InterruptedException + { + //if it was due to priority then remove it now + if (turn == MY_TURN_DUE_TO_PRIORITY) + { + logger.info("Remove current host from priority list"); + AutoRepairUtils.removePriorityStatus(repairType, myId); + } + long repairScheduleElapsedInMillis = timeFunc.get() - startTimeInMillis; + if (repairScheduleElapsedInMillis < SLEEP_IF_REPAIR_FINISHES_QUICKLY.toMilliseconds()) + { + //If repair finished quickly, happens for Cassndra cluster with empty (or tiny) data, in such cases, + //wait for some duration so that the JMX metrics can detect the repairInProgress + logger.info("Wait for {}ms for repair type {}.", SLEEP_IF_REPAIR_FINISHES_QUICKLY.toMilliseconds() - repairScheduleElapsedInMillis, repairType); + Thread.sleep(SLEEP_IF_REPAIR_FINISHES_QUICKLY.toMilliseconds() - repairScheduleElapsedInMillis); + } + repairState.setFailedTokenRangesCount(collectedRepairStats.failedTokenRanges); + repairState.setSucceededTokenRangesCount(collectedRepairStats.succeededTokenRanges); + repairState.setSkippedTokenRangesCount(collectedRepairStats.skippedTokenRanges); + repairState.setSkippedTablesCount(collectedRepairStats.skippedTables); + repairState.setNodeRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - startTimeInMillis)); + long timeInHours = TimeUnit.SECONDS.toHours(repairState.getNodeRepairTimeInSec()); + logger.info("Local {} repair time {} hour(s), stats: repairKeyspaceCount {}, " + + "repairTokenRangesSuccessCount {}, repairTokenRangesFailureCount {}, " + + "repairTokenRangesSkipCount {}, repairTablesSkipCount {}", repairType, timeInHours, repairState.getRepairKeyspaceCount(), + repairState.getSucceededTokenRangesCount(), repairState.getFailedTokenRangesCount(), + repairState.getSkippedTokenRangesCount(), repairState.getSkippedTablesCount()); + if (repairState.getLastRepairTime() != 0) + { + repairState.setClusterRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - + repairState.getLastRepairTime())); + logger.info("Cluster repair time for repair type {}: {} day(s)", repairType, + TimeUnit.SECONDS.toDays(repairState.getClusterRepairTimeInSec())); + } + repairState.setLastRepairTime(timeFunc.get()); + repairState.setRepairInProgress(false); + + AutoRepairUtils.updateFinishAutoRepairHistory(repairType, myId, timeFunc.get()); + } + + public AutoRepairState getRepairState(AutoRepairConfig.RepairType repairType) + { + return repairStates.get(repairType); + } + + private void soakAfterRepair(long startTimeMilis, long minDurationMilis) + { + long currentTime = timeFunc.get(); + long timeElapsed = currentTime - startTimeMilis; + if (timeElapsed < minDurationMilis) + { + long timeToSoak = minDurationMilis - timeElapsed; + logger.info("Soaking for {} ms after repair", timeToSoak); + sleepFunc.accept(timeToSoak, TimeUnit.MILLISECONDS); + } + } + + static class CollectedRepairStats + { + int failedTokenRanges = 0; + int succeededTokenRanges = 0; + int skippedTokenRanges = 0; + int skippedTables = 0; + } + + @VisibleForTesting + protected static class RepairProgressListener implements ProgressListener + { + private final AutoRepairConfig.RepairType repairType; + @VisibleForTesting + protected boolean success; + @VisibleForTesting + protected final Condition condition = newOneTimeCondition(); + + public RepairProgressListener(AutoRepairConfig.RepairType repairType) + { + this.repairType = repairType; + } + + public void await(DurationSpec.IntSecondsBound repairSessionTimeout) throws InterruptedException + { + //if for some reason we don't hear back on repair progress for sometime + if (!condition.await(repairSessionTimeout.to(TimeUnit.SECONDS), TimeUnit.SECONDS)) + { + success = false; + } + } + + public boolean isSuccess() + { + return success; + } + + @Override + public void progress(String tag, ProgressEvent event) + { + ProgressEventType type = event.getType(); + String message = String.format("[%s] %s", format.format(timeFunc.get()), event.getMessage()); + if (type == ProgressEventType.ERROR) + { + logger.error("Repair failure for repair {}: {}", repairType.toString(), message); + success = false; + condition.signalAll(); + } + if (type == ProgressEventType.PROGRESS) + { + message = message + " (progress: " + (int) event.getProgressPercentage() + "%)"; + logger.debug("Repair progress for repair {}: {}", repairType.toString(), message); + } + if (type == ProgressEventType.COMPLETE) + { + logger.debug("Repair completed for repair {}: {}", repairType.toString(), message); + success = true; + condition.signalAll(); + } + } + } + + public synchronized void shutdownBlocking() throws ExecutionException, InterruptedException + { + if (!isSetupDone) + { + // By default, executors within AutoRepair are not initialized as the feature is opt-in. + // If the AutoRepair has not been set up, then there is no need to worry about shutting it down + return; + } + if (isShutDown) + { + throw new IllegalStateException("AutoRepair has already been shut down"); + } + isShutDown = true; + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + repairRunnableExecutors.get(repairType).shutdown(); + repairExecutors.get(repairType).shutdown(); + } + logger.info("Paused AutoRepair"); + } + + public Map getRepairExecutors() + { + return repairExecutors; + } + + public Map getRepairRunnableExecutors() + { + return repairRunnableExecutors; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java new file mode 100644 index 000000000000..0998a4a37eb1 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -0,0 +1,615 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.io.Serializable; +import java.util.Collections; +import java.util.EnumMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Function; + +import javax.annotation.Nonnull; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; + +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.config.ParameterizedClass; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.utils.FBUtilities; + +/** + * Defines configurations for AutoRepair. + */ +public class AutoRepairConfig implements Serializable +{ + // Enable/Disable the auto-repair scheduler. + // If set to false, the scheduler thread will not be started. + // If set to true, the repair scheduler thread will be created. The thread will + // check for secondary configuration available for each repair type (full, incremental, + // and preview_repaired), and based on that, it will schedule repairs. + public volatile Boolean enabled; + // Time interval between successive checks to see if ongoing repairs are complete or if it is time to schedule + // repairs. + public final DurationSpec.IntSecondsBound repair_check_interval = new DurationSpec.IntSecondsBound("5m"); + // The scheduler needs to adjust its order when nodes leave the ring. Deleted hosts are tracked in metadata + // for a specified duration to ensure they are indeed removed before adjustments are made to the schedule. + public volatile DurationSpec.IntSecondsBound history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound("2h"); + // Minimum duration for the execution of a single repair task. This prevents the scheduler from overwhelming + // the node by scheduling too many repair tasks in a short period of time. + public volatile DurationSpec.LongSecondsBound repair_task_min_duration = new DurationSpec.LongSecondsBound("5s"); + // by default repair is disabled if there are mixed major versions detected, but you can enable it using this flag + public volatile boolean mixed_major_version_repair_enabled = false; + + // global_settings overides Options.defaultOptions for all repair types + public volatile Options global_settings; + + public static final Class DEFAULT_SPLITTER = RepairTokenRangeSplitter.class; + + // make transient so gets consturcted in the implementation. + private final transient Map tokenRangeSplitters = new EnumMap<>(RepairType.class); + + public enum RepairType implements Serializable + { + FULL, + INCREMENTAL, + PREVIEW_REPAIRED; + + private final String configName; + + RepairType() + { + this.configName = name().toLowerCase(); + } + + /** + * @return Format of the repair type as it should be represented in configuration. + * Canonically this is the enum name in lowerCase. + */ + public String getConfigName() + { + return configName; + } + + public static AutoRepairState getAutoRepairState(RepairType repairType, AutoRepairConfig config) + { + switch (repairType) + { + case FULL: + return new FullRepairState(config); + case INCREMENTAL: + return new IncrementalRepairState(config); + case PREVIEW_REPAIRED: + return new PreviewRepairedState(config); + } + + throw new IllegalArgumentException("Invalid repair type: " + repairType); + } + + /** + * Case-insensitive parsing of the repair type string into {@link RepairType} + * + * @param repairTypeStr the repair type string + * @return the {@link RepairType} represented by the {@code repairTypeStr} string + * @throws IllegalArgumentException when the repair type string does not match any repair type + */ + public static RepairType parse(String repairTypeStr) + { + return RepairType.valueOf(Objects.requireNonNull(repairTypeStr, "repairTypeStr cannot be null").toUpperCase()); + } + } + + // repair_type_overrides overrides the global_settings for a specific repair type. String used as key instead + // of enum to allow lower case key in yaml. + public volatile ConcurrentMap repair_type_overrides = Maps.newConcurrentMap(); + + public AutoRepairConfig() + { + this(false); + } + + public AutoRepairConfig(boolean enabled) + { + this.enabled = enabled; + global_settings = Options.getDefaultOptions(); + } + + public DurationSpec.IntSecondsBound getRepairCheckInterval() + { + return repair_check_interval; + } + + public boolean isAutoRepairSchedulingEnabled() + { + return enabled; + } + + @VisibleForTesting + public void setAutoRepairSchedulingEnabled(boolean enabled) + { + this.enabled = enabled; + } + + public boolean isMixedMajorVersionRepairEnabled() + { + return mixed_major_version_repair_enabled; + } + + public DurationSpec.IntSecondsBound getAutoRepairHistoryClearDeleteHostsBufferInterval() + { + return history_clear_delete_hosts_buffer_interval; + } + + public void startScheduler() + { + enabled = true; + AutoRepair.instance.setup(); + } + + public void setAutoRepairHistoryClearDeleteHostsBufferInterval(String duration) + { + history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound(duration); + } + + public DurationSpec.LongSecondsBound getRepairTaskMinDuration() + { + return repair_task_min_duration; + } + + public void setRepairTaskMinDuration(String duration) + { + repair_task_min_duration = new DurationSpec.LongSecondsBound(duration); + } + + public boolean isAutoRepairEnabled(RepairType repairType) + { + return enabled && applyOverrides(repairType, opt -> opt.enabled); + } + + public void setAutoRepairEnabled(RepairType repairType, boolean enabled) + { + getOptions(repairType).enabled = enabled; + } + + public void setRepairByKeyspace(RepairType repairType, boolean repairByKeyspace) + { + getOptions(repairType).repair_by_keyspace = repairByKeyspace; + } + + public boolean getRepairByKeyspace(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.repair_by_keyspace); + } + + public int getRepairThreads(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.number_of_repair_threads); + } + + public void setRepairThreads(RepairType repairType, int repairThreads) + { + getOptions(repairType).number_of_repair_threads = repairThreads; + } + + public DurationSpec.IntSecondsBound getRepairMinInterval(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.min_repair_interval); + } + + public void setRepairMinInterval(RepairType repairType, String minRepairInterval) + { + getOptions(repairType).min_repair_interval = new DurationSpec.IntSecondsBound(minRepairInterval); + } + + public int getRepairSSTableCountHigherThreshold(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.sstable_upper_threshold); + } + + public void setRepairSSTableCountHigherThreshold(RepairType repairType, int sstableHigherThreshold) + { + getOptions(repairType).sstable_upper_threshold = sstableHigherThreshold; + } + + public DurationSpec.IntSecondsBound getAutoRepairTableMaxRepairTime(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.table_max_repair_time); + } + + public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime) + { + getOptions(repairType).table_max_repair_time = new DurationSpec.IntSecondsBound(autoRepairTableMaxRepairTime); + } + + public Set getIgnoreDCs(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.ignore_dcs); + } + + public void setIgnoreDCs(RepairType repairType, Set ignoreDCs) + { + getOptions(repairType).ignore_dcs = ignoreDCs; + } + + public boolean getRepairPrimaryTokenRangeOnly(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.repair_primary_token_range_only); + } + + public void setRepairPrimaryTokenRangeOnly(RepairType repairType, boolean primaryTokenRangeOnly) + { + getOptions(repairType).repair_primary_token_range_only = primaryTokenRangeOnly; + } + + public int getParallelRepairPercentage(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.parallel_repair_percentage); + } + + public void setParallelRepairPercentage(RepairType repairType, int percentage) + { + getOptions(repairType).parallel_repair_percentage = percentage; + } + + public int getParallelRepairCount(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.parallel_repair_count); + } + + public void setParallelRepairCount(RepairType repairType, int count) + { + getOptions(repairType).parallel_repair_count = count; + } + + public boolean getAllowParallelReplicaRepair(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.allow_parallel_replica_repair); + } + + public void setAllowParallelReplicaRepair(RepairType repairType, boolean enabled) + { + getOptions(repairType).allow_parallel_replica_repair = enabled; + } + + public boolean getAllowParallelReplicaRepairAcrossSchedules(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.allow_parallel_replica_repair_across_schedules); + } + + public void setAllowParallelReplicaRepairAcrossSchedules(RepairType repairType, boolean enabled) + { + getOptions(repairType).allow_parallel_replica_repair_across_schedules = enabled; + } + + public boolean getMaterializedViewRepairEnabled(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.materialized_view_repair_enabled); + } + + public void setMaterializedViewRepairEnabled(RepairType repairType, boolean enabled) + { + getOptions(repairType).materialized_view_repair_enabled = enabled; + } + + public void setForceRepairNewNode(RepairType repairType, boolean forceRepairNewNode) + { + getOptions(repairType).force_repair_new_node = forceRepairNewNode; + } + + public boolean getForceRepairNewNode(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.force_repair_new_node); + } + + public ParameterizedClass getTokenRangeSplitter(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.token_range_splitter); + } + + public IAutoRepairTokenRangeSplitter getTokenRangeSplitterInstance(RepairType repairType) + { + return tokenRangeSplitters.computeIfAbsent(repairType, + key -> newAutoRepairTokenRangeSplitter(key, getTokenRangeSplitter(key))); + } + + public void setInitialSchedulerDelay(RepairType repairType, String initialSchedulerDelay) + { + getOptions(repairType).initial_scheduler_delay = new DurationSpec.IntSecondsBound(initialSchedulerDelay); + } + + public DurationSpec.IntSecondsBound getInitialSchedulerDelay(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.initial_scheduler_delay); + } + + public DurationSpec.IntSecondsBound getRepairSessionTimeout(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.repair_session_timeout); + } + + public void setRepairSessionTimeout(RepairType repairType, String repairSessionTimeout) + { + getOptions(repairType).repair_session_timeout = new DurationSpec.IntSecondsBound(repairSessionTimeout); + } + + public int getRepairMaxRetries(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.repair_max_retries); + } + + public void setRepairMaxRetries(RepairType repairType, int maxRetries) + { + getOptions(repairType).repair_max_retries = maxRetries; + } + + public DurationSpec.LongSecondsBound getRepairRetryBackoff(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.repair_retry_backoff); + } + + public void setRepairRetryBackoff(RepairType repairType, String interval) + { + getOptions(repairType).repair_retry_backoff = new DurationSpec.LongSecondsBound(interval); + } + + public boolean getMixedMajorVersionRepairEnabled() + { + return this.mixed_major_version_repair_enabled; + } + + public void setMixedMajorVersionRepairEnabled(boolean enabled) + { + this.mixed_major_version_repair_enabled = enabled; + } + + @VisibleForTesting + static IAutoRepairTokenRangeSplitter newAutoRepairTokenRangeSplitter(RepairType repairType, ParameterizedClass parameterizedClass) throws ConfigurationException + { + try + { + Class tokenRangeSplitterClass; + final String className; + if (parameterizedClass.class_name != null && !parameterizedClass.class_name.isEmpty()) + { + className = parameterizedClass.class_name.contains(".") ? + parameterizedClass.class_name : + "org.apache.cassandra.repair.autorepair." + parameterizedClass.class_name; + tokenRangeSplitterClass = FBUtilities.classForName(className, "token_range_splitter"); + } + else + { + // If token_range_splitter.class_name is not defined, just use default, this is for convenience. + tokenRangeSplitterClass = AutoRepairConfig.DEFAULT_SPLITTER; + } + try + { + Map parameters = parameterizedClass.parameters != null ? parameterizedClass.parameters : Collections.emptyMap(); + // first attempt to initialize with RepairType and Map arguments. + return tokenRangeSplitterClass.getConstructor(RepairType.class, Map.class).newInstance(repairType, parameters); + } + catch (NoSuchMethodException nsme) + { + // fall back on no argument constructor. + return tokenRangeSplitterClass.getConstructor().newInstance(); + } + } + catch (Exception ex) + { + throw new ConfigurationException("Unable to create instance of IAutoRepairTokenRangeSplitter", ex); + } + } + + // Options configures auto-repair behavior for a given repair type. + // All fields can be modified dynamically. + public static class Options implements Serializable + { + // defaultOptions defines the default auto-repair behavior when no overrides are defined + @VisibleForTesting + private static Map defaultOptions; + + private static Map initializeDefaultOptions() + { + Map options = new EnumMap<>(AutoRepairConfig.RepairType.class); + options.put(AutoRepairConfig.RepairType.FULL, getDefaultOptions()); + options.put(RepairType.INCREMENTAL, getDefaultOptions()); + options.put(RepairType.PREVIEW_REPAIRED, getDefaultOptions()); + + return options; + } + + public static Map getDefaultOptionsMap() + { + if (defaultOptions == null) + { + synchronized (AutoRepairConfig.class) + { + if (defaultOptions == null) + { + defaultOptions = initializeDefaultOptions(); + } + } + } + return defaultOptions; + } + + public Options() + { + } + + @VisibleForTesting + protected static Options getDefaultOptions() + { + Options opts = new Options(); + + opts.enabled = false; + opts.repair_by_keyspace = true; + opts.number_of_repair_threads = 1; + opts.parallel_repair_count = 3; + opts.parallel_repair_percentage = 3; + opts.allow_parallel_replica_repair = false; + opts.allow_parallel_replica_repair_across_schedules = true; + opts.sstable_upper_threshold = 50000; + opts.ignore_dcs = new HashSet<>(); + opts.repair_primary_token_range_only = true; + opts.force_repair_new_node = false; + opts.table_max_repair_time = new DurationSpec.IntSecondsBound("6h"); + opts.materialized_view_repair_enabled = false; + opts.token_range_splitter = new ParameterizedClass(DEFAULT_SPLITTER.getName(), Collections.emptyMap()); + opts.initial_scheduler_delay = new DurationSpec.IntSecondsBound("5m"); + opts.repair_session_timeout = new DurationSpec.IntSecondsBound("3h"); + opts.min_repair_interval = new DurationSpec.IntSecondsBound("24h"); + + return opts; + } + + // Enable/Disable full or incremental or previewed_repair auto repair + public volatile Boolean enabled; + // If true, attempts to group tables in the same keyspace into one repair; otherwise, each table is repaired + // individually. + public volatile Boolean repair_by_keyspace; + // Number of threads to use for each repair job scheduled by the scheduler. Similar to the -j option in nodetool + // repair. + public volatile Integer number_of_repair_threads; + // Number of nodes running repair in parallel. If parallel_repair_percentage is set, the larger value is used. + public volatile Integer parallel_repair_count; + // Percentage of nodes in the cluster running repair in parallel. If parallel_repair_count is set, the larger value + // is used. Recommendation is that the repair cycle on the cluster should finish within gc_grace_seconds. + public volatile Integer parallel_repair_percentage; + // Whether to allow a node to take its turn running repair while one or more of its replicas are running repair. + // Defaults to false, as running repairs concurrently on replicas can increase load and also cause + // anticompaction conflicts while running incremental repair. + public volatile Boolean allow_parallel_replica_repair; + // An addition to allow_parallel_replica_repair that also blocks repairs when replicas (including this node itself) + // are repairing in any schedule. For example, if a replica is executing full repairs, a value of false will + // prevent starting incremental repairs for this node. Defaults to true and is only evaluated when + // allow_parallel_replica_repair is false. + public volatile Boolean allow_parallel_replica_repair_across_schedules; + // Threshold to skip repairing tables with too many SSTables. Defaults to 10,000 SSTables to avoid penalizing good + // tables. + public volatile Integer sstable_upper_threshold; + // Minimum duration between repairing the same node again. This is useful for tiny clusters, such as + // clusters with 5 nodes that finish repairs quickly. The default is 24 hours. This means that if the scheduler + // completes one round on all nodes in less than 24 hours, it will not start a new repair round on a given node + // until 24 hours have passed since the last repair. + public volatile DurationSpec.IntSecondsBound min_repair_interval; + // Avoid running repairs in specific data centers. By default, repairs run in all data centers. Specify data + // centers to exclude in this list. Note that repair sessions will still consider all replicas from excluded + // data centers. Useful if you have keyspaces that are not replicated in certain data centers, and you want to + // not run repair schedule in certain data centers. + public volatile Set ignore_dcs; + // Repair only the primary ranges owned by a node. Equivalent to the -pr option in nodetool repair. Defaults + // to true. General advice is to keep this true. + public volatile Boolean repair_primary_token_range_only; + // Force immediate repair on new nodes after they join the ring. + public volatile Boolean force_repair_new_node; + // Maximum time allowed for repairing one table on a given node. If exceeded, the repair proceeds to the + // next table. + public volatile DurationSpec.IntSecondsBound table_max_repair_time; + // Repairs materialized views if true. + public volatile Boolean materialized_view_repair_enabled; + /** + * Splitter implementation to use for generating repair assignments. + *

+ * The default is {@link RepairTokenRangeSplitter}. The class should implement {@link IAutoRepairTokenRangeSplitter} + * and have a constructor accepting ({@link RepairType}, {@link java.util.Map}) + */ + public volatile ParameterizedClass token_range_splitter; + // After a node restart, wait for this much delay before scheduler starts running repair; this is to avoid starting repair immediately after a node restart. + public volatile DurationSpec.IntSecondsBound initial_scheduler_delay; + // Timeout for retrying stuck repair sessions. + public volatile DurationSpec.IntSecondsBound repair_session_timeout; + // Maximum number of retries for a repair session. + public volatile Integer repair_max_retries = 3; + // Backoff time before retrying a repair session. + public volatile DurationSpec.LongSecondsBound repair_retry_backoff = new DurationSpec.LongSecondsBound("30s"); + + public String toString() + { + return "Options{" + + "enabled=" + enabled + + ", repair_by_keyspace=" + repair_by_keyspace + + ", number_of_repair_threads=" + number_of_repair_threads + + ", parallel_repair_count=" + parallel_repair_count + + ", parallel_repair_percentage=" + parallel_repair_percentage + + ", allow_parallel_replica_repair=" + allow_parallel_replica_repair + + ", allow_parallel_replica_repair_across_schedules=" + allow_parallel_replica_repair_across_schedules + + ", sstable_upper_threshold=" + sstable_upper_threshold + + ", min_repair_interval=" + min_repair_interval + + ", ignore_dcs=" + ignore_dcs + + ", repair_primary_token_range_only=" + repair_primary_token_range_only + + ", force_repair_new_node=" + force_repair_new_node + + ", table_max_repair_time=" + table_max_repair_time + + ", materialized_view_repair_enabled=" + materialized_view_repair_enabled + + ", token_range_splitter=" + token_range_splitter + + ", intial_scheduler_delay=" + initial_scheduler_delay + + ", repair_session_timeout=" + repair_session_timeout + + '}'; + } + } + + @Nonnull + protected Options getOptions(RepairType repairType) + { + return repair_type_overrides.computeIfAbsent(repairType.getConfigName(), k -> new Options()); + } + + private static T getOverride(Options options, Function optionSupplier) + { + return options != null ? optionSupplier.apply(options) : null; + } + + @VisibleForTesting + protected T applyOverrides(RepairType repairType, Function optionSupplier) + { + // Check option by repair type first + Options repairTypeOverrides = getOptions(repairType); + T val = optionSupplier.apply(repairTypeOverrides); + + if (val != null) + return val; + + // Check option in global settings + if (global_settings != null) + { + val = getOverride(global_settings, optionSupplier); + + if (val != null) + return val; + } + + // Otherwise check defaults + return getOverride(Options.getDefaultOptionsMap().get(repairType), optionSupplier); + } + + public String toString() + { + return "AutoRepairConfig{" + + "enabled=" + enabled + + ", repair_check_interval=" + repair_check_interval + + ", history_clear_delete_hosts_buffer_interval=" + history_clear_delete_hosts_buffer_interval + + ", repair_task_min_duration=" + repair_task_min_duration + + ", global_settings=" + global_settings + + ", repair_type_overrides=" + repair_type_overrides + + "}"; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java new file mode 100644 index 000000000000..bf319b0fd00e --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -0,0 +1,386 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.view.TableViews; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; +import org.apache.cassandra.repair.RepairParallelism; +import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.repair.messages.RepairOption; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.utils.Clock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.text.SimpleDateFormat; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +/** + * AutoRepairState represents the state of automated repair for a given repair type. + */ +public abstract class AutoRepairState +{ + protected static final Logger logger = LoggerFactory.getLogger(AutoRepairState.class); + private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS"); + @VisibleForTesting + protected static Supplier timeFunc = Clock.Global::currentTimeMillis; + + @VisibleForTesting + protected final RepairType repairType; + @VisibleForTesting + protected AutoRepairConfig config; + @VisibleForTesting + protected int totalTablesConsideredForRepair = 0; + @VisibleForTesting + protected long lastRepairTimeInMs; + @VisibleForTesting + protected int nodeRepairTimeInSec = 0; + @VisibleForTesting + protected int clusterRepairTimeInSec = 0; + @VisibleForTesting + protected boolean repairInProgress = false; + @VisibleForTesting + protected int repairKeyspaceCount = 0; + @VisibleForTesting + protected int totalMVTablesConsideredForRepair = 0; + @VisibleForTesting + protected int totalDisabledTablesRepairCount = 0; + @VisibleForTesting + protected int failedTokenRangesCount = 0; + @VisibleForTesting + protected int succeededTokenRangesCount = 0; + @VisibleForTesting + protected int skippedTokenRangesCount = 0; + @VisibleForTesting + protected int skippedTablesCount = 0; + @VisibleForTesting + protected long totalBytesToRepair = 0; + @VisibleForTesting + protected long bytesAlreadyRepaired = 0; + @VisibleForTesting + protected int totalKeyspaceRepairPlansToRepair = 0; + @VisibleForTesting + protected int keyspaceRepairPlansAlreadyRepaired = 0; + @VisibleForTesting + protected AutoRepairHistory longestUnrepairedNode; + protected final AutoRepairMetrics metrics; + + protected AutoRepairState(RepairType repairType, AutoRepairConfig config) + { + metrics = AutoRepairMetricsManager.getMetrics(repairType); + this.repairType = repairType; + this.config = config; + } + + public abstract RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly); + + protected RepairCoordinator getRepairRunnable(String keyspace, RepairOption options) + { + return new RepairCoordinator(StorageService.instance, StorageService.nextRepairCommand.incrementAndGet(), + options, keyspace); + } + + public void updateRepairScheduleStatistics(List repairPlans) + { + setTotalBytesToRepair(repairPlans.stream(). + flatMap(repairPlan -> repairPlan.getKeyspaceRepairPlans(). + stream()).mapToLong(KeyspaceRepairPlan::getEstimatedBytes).sum()); + setTotalKeyspaceRepairPlansToRepair(repairPlans.stream().mapToInt(repairPlan -> repairPlan.getKeyspaceRepairPlans().size()).sum()); + } + + public long getLastRepairTime() + { + return lastRepairTimeInMs; + } + + public void setTotalTablesConsideredForRepair(int count) + { + totalTablesConsideredForRepair = count; + } + + public int getTotalTablesConsideredForRepair() + { + return totalTablesConsideredForRepair; + } + + public void setLastRepairTime(long lastRepairTime) + { + lastRepairTimeInMs = lastRepairTime; + } + + public int getClusterRepairTimeInSec() + { + return clusterRepairTimeInSec; + } + + public int getNodeRepairTimeInSec() + { + return nodeRepairTimeInSec; + } + + public void setRepairInProgress(boolean repairInProgress) + { + this.repairInProgress = repairInProgress; + } + + public boolean isRepairInProgress() + { + return repairInProgress; + } + + public int getLongestUnrepairedSec() + { + if (longestUnrepairedNode == null) + { + return 0; + } + return (int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - longestUnrepairedNode.getLastRepairFinishTime()); + } + + public void setTotalMVTablesConsideredForRepair(int count) + { + totalMVTablesConsideredForRepair = count; + } + + public int getTotalMVTablesConsideredForRepair() + { + return totalMVTablesConsideredForRepair; + } + + public void setNodeRepairTimeInSec(int elapsed) + { + nodeRepairTimeInSec = elapsed; + } + + public void setClusterRepairTimeInSec(int seconds) + { + clusterRepairTimeInSec = seconds; + } + + public void setRepairKeyspaceCount(int count) + { + repairKeyspaceCount = count; + } + + public int getRepairKeyspaceCount() + { + return repairKeyspaceCount; + } + + public void setLongestUnrepairedNode(AutoRepairHistory longestUnrepairedNode) + { + this.longestUnrepairedNode = longestUnrepairedNode; + } + + public void setFailedTokenRangesCount(int count) + { + failedTokenRangesCount = count; + } + + public int getFailedTokenRangesCount() + { + return failedTokenRangesCount; + } + + public void setSucceededTokenRangesCount(int count) + { + succeededTokenRangesCount = count; + } + + public int getSucceededTokenRangesCount() + { + return succeededTokenRangesCount; + } + + public void setSkippedTokenRangesCount(int count) + { + skippedTokenRangesCount = count; + } + + public int getSkippedTokenRangesCount() + { + return skippedTokenRangesCount; + } + + public void setSkippedTablesCount(int count) + { + skippedTablesCount = count; + } + + public int getSkippedTablesCount() + { + return skippedTablesCount; + } + + public void recordTurn(AutoRepairUtils.RepairTurn turn) + { + metrics.recordTurn(turn); + } + + public void setTotalDisabledTablesRepairCount(int count) + { + totalDisabledTablesRepairCount = count; + } + + public int getTotalDisabledTablesRepairCount() + { + return totalDisabledTablesRepairCount; + } + + public void setTotalBytesToRepair(long totalBytesToRepair) + { + this.totalBytesToRepair = totalBytesToRepair; + } + + public long getTotalBytesToRepair() + { + return totalBytesToRepair; + } + + public void setBytesAlreadyRepaired(long bytesAlreadyRepaired) + { + this.bytesAlreadyRepaired = bytesAlreadyRepaired; + } + + public long getBytesAlreadyRepaired() + { + return bytesAlreadyRepaired; + } + + public void setTotalKeyspaceRepairPlansToRepair(int totalKeyspaceRepairPlansToRepair) + { + this.totalKeyspaceRepairPlansToRepair = totalKeyspaceRepairPlansToRepair; + } + + public int getTotalKeyspaceRepairPlansToRepair() + { + return totalKeyspaceRepairPlansToRepair; + } + + public void setKeyspaceRepairPlansAlreadyRepaired(int keyspaceRepairPlansAlreadyRepaired) + { + this.keyspaceRepairPlansAlreadyRepaired = keyspaceRepairPlansAlreadyRepaired; + } + + public int getKeyspaceRepairPlansAlreadyRepaired() + { + return keyspaceRepairPlansAlreadyRepaired; + } +} + +class PreviewRepairedState extends AutoRepairState +{ + public PreviewRepairedState(AutoRepairConfig config) + { + super(RepairType.PREVIEW_REPAIRED, config); + } + + @Override + public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) + { + RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, false, false, + AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, + !ranges.isEmpty(), false, false, PreviewKind.REPAIRED, false, true, false, false); + + option.getColumnFamilies().addAll(tables); + + return getRepairRunnable(keyspace, option); + } +} + +class IncrementalRepairState extends AutoRepairState +{ + public IncrementalRepairState(AutoRepairConfig config) + { + super(RepairType.INCREMENTAL, config); + } + + @Override + public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) + { + RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, true, false, + AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, + !ranges.isEmpty(), false, false, PreviewKind.NONE, true, true, false, false); + + option.getColumnFamilies().addAll(filterOutUnsafeTables(keyspace, tables)); + + return getRepairRunnable(keyspace, option); + } + + @VisibleForTesting + protected List filterOutUnsafeTables(String keyspaceName, List tables) + { + Keyspace keyspace = Keyspace.open(keyspaceName); + + return tables.stream() + .filter(table -> { + ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(table); + TableViews views = keyspace.viewManager.forTable(cfs.metadata().id); + if (views != null && !views.isEmpty()) + { + logger.debug("Skipping incremental repair for {}.{} as it has materialized views", keyspaceName, table); + return false; + } + + if (cfs.metadata().params != null && cfs.metadata().params.cdc) + { + logger.debug("Skipping incremental repair for {}.{} as it has CDC enabled", keyspaceName, table); + return false; + } + + return true; + }).collect(Collectors.toList()); + } +} + +class FullRepairState extends AutoRepairState +{ + public FullRepairState(AutoRepairConfig config) + { + super(RepairType.FULL, config); + } + + @Override + public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) + { + RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, false, false, + AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, + !ranges.isEmpty(), false, false, PreviewKind.NONE, true, true, false, false); + + option.getColumnFamilies().addAll(tables); + + return getRepairRunnable(keyspace, option); + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java new file mode 100644 index 000000000000..183b19ced7c8 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -0,0 +1,1419 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.repair.autorepair; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.TreeSet; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import com.clearspring.analytics.stream.cardinality.CardinalityMergeException; +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus; +import com.clearspring.analytics.stream.cardinality.ICardinality; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Splitter; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.metadata.CompactionMetadata; +import org.apache.cassandra.io.sstable.metadata.MetadataType; +import org.apache.cassandra.locator.EndpointsByRange; +import org.apache.cassandra.locator.EndpointsForRange; +import org.apache.cassandra.locator.LocalStrategy; + +import org.apache.cassandra.utils.CassandraVersion; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.cql3.statements.ModificationStatement; +import org.apache.cassandra.cql3.statements.SelectStatement; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.NetworkTopologyStrategy; +import org.apache.cassandra.locator.RangesAtEndpoint; +import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SystemDistributedKeyspace; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.ViewMetadata; +import org.apache.cassandra.serializers.SetSerializer; +import org.apache.cassandra.serializers.UUIDSerializer; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.transport.Dispatcher; +import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.utils.NoSpamLogger; +import org.apache.cassandra.utils.concurrent.Refs; + +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; + +/** + * This class serves as a utility class for AutoRepair. It contains various helper APIs + * to store/retrieve repair status, decide whose turn is next, etc. + */ +public class AutoRepairUtils +{ + private static final Logger logger = LoggerFactory.getLogger(AutoRepairUtils.class); + static final String COL_REPAIR_TYPE = "repair_type"; + static final String COL_HOST_ID = "host_id"; + static final String COL_REPAIR_START_TS = "repair_start_ts"; + static final String COL_REPAIR_FINISH_TS = "repair_finish_ts"; + static final String COL_REPAIR_PRIORITY = "repair_priority"; + static final String COL_DELETE_HOSTS = "delete_hosts"; // this set stores the host ids which think the row should be deleted + static final String COL_REPAIR_TURN = "repair_turn"; // this record the last repair turn. Normal turn or turn due to priority + static final String COL_DELETE_HOSTS_UPDATE_TIME = "delete_hosts_update_time"; // the time when delete hosts are upated + static final String COL_FORCE_REPAIR = "force_repair"; // if set to true, the node will do non-primary range rapair + + final static String SELECT_REPAIR_HISTORY = String.format( + "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE); + final static String SELECT_REPAIR_PRIORITY = String.format( + "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); + final static String DEL_REPAIR_PRIORITY = String.format( + "DELETE %s[?] FROM %s.%s WHERE %s = ?", COL_REPAIR_PRIORITY, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); + final static String ADD_PRIORITY_HOST = String.format( + "UPDATE %s.%s SET %s = %s + ? WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_TYPE); + + final static String INSERT_NEW_REPAIR_HISTORY = String.format( + "INSERT INTO %s.%s (%s, %s, %s, %s, %s, %s) values (?, ? ,?, ?, {}, ?) IF NOT EXISTS", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, + COL_HOST_ID, COL_REPAIR_START_TS, COL_REPAIR_FINISH_TS, COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME); + + final static String ADD_HOST_ID_TO_DELETE_HOSTS = String.format( + "UPDATE %s.%s SET %s = %s + ?, %s = ? WHERE %s = ? AND %s = ? IF EXISTS" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, + COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME, COL_REPAIR_TYPE, COL_HOST_ID); + + final static String DEL_AUTO_REPAIR_HISTORY = String.format( + "DELETE FROM %s.%s WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, + COL_HOST_ID); + + final static String RECORD_START_REPAIR_HISTORY = String.format( + "UPDATE %s.%s SET %s= ?, repair_turn = ? WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_START_TS, + COL_REPAIR_TYPE, COL_HOST_ID); + + final static String RECORD_FINISH_REPAIR_HISTORY = String.format( + "UPDATE %s.%s SET %s= ?, %s=false WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_FINISH_TS, + COL_FORCE_REPAIR, COL_REPAIR_TYPE, COL_HOST_ID); + + final static String CLEAR_DELETE_HOSTS = String.format( + "UPDATE %s.%s SET %s= {} WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, + COL_REPAIR_TYPE, COL_HOST_ID); + + final static String SET_FORCE_REPAIR = String.format( + "UPDATE %s.%s SET %s=true WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_FORCE_REPAIR, + COL_REPAIR_TYPE, COL_HOST_ID); + + final static String SELECT_LAST_REPAIR_TIME_FOR_NODE = String.format( + "SELECT %s FROM %s.%s WHERE %s = ? AND %s = ?", COL_REPAIR_FINISH_TS, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); + + static ModificationStatement delStatementRepairHistory; + static SelectStatement selectStatementRepairHistory; + static ModificationStatement delStatementPriorityStatus; + static SelectStatement selectStatementRepairPriority; + static SelectStatement selectLastRepairTimeForNode; + static ModificationStatement addPriorityHost; + static ModificationStatement insertNewRepairHistoryStatement; + static ModificationStatement recordStartRepairHistoryStatement; + static ModificationStatement recordFinishRepairHistoryStatement; + static ModificationStatement addHostIDToDeleteHostsStatement; + static ModificationStatement clearDeleteHostsStatement; + static ModificationStatement setForceRepairStatement; + static ConsistencyLevel internalQueryCL; + + public enum RepairTurn + { + MY_TURN, + NOT_MY_TURN, + MY_TURN_DUE_TO_PRIORITY, + MY_TURN_FORCE_REPAIR + } + + public static void setup() + { + selectStatementRepairHistory = (SelectStatement) QueryProcessor.getStatement(SELECT_REPAIR_HISTORY, ClientState + .forInternalCalls()); + selectStatementRepairPriority = (SelectStatement) QueryProcessor.getStatement(SELECT_REPAIR_PRIORITY, ClientState + .forInternalCalls()); + selectLastRepairTimeForNode = (SelectStatement) QueryProcessor.getStatement(SELECT_LAST_REPAIR_TIME_FOR_NODE, ClientState + .forInternalCalls()); + delStatementPriorityStatus = (ModificationStatement) QueryProcessor.getStatement(DEL_REPAIR_PRIORITY, ClientState + .forInternalCalls()); + addPriorityHost = (ModificationStatement) QueryProcessor.getStatement(ADD_PRIORITY_HOST, ClientState + .forInternalCalls()); + insertNewRepairHistoryStatement = (ModificationStatement) QueryProcessor.getStatement(INSERT_NEW_REPAIR_HISTORY, ClientState + .forInternalCalls()); + recordStartRepairHistoryStatement = (ModificationStatement) QueryProcessor.getStatement(RECORD_START_REPAIR_HISTORY, ClientState + .forInternalCalls()); + recordFinishRepairHistoryStatement = (ModificationStatement) QueryProcessor.getStatement(RECORD_FINISH_REPAIR_HISTORY, ClientState + .forInternalCalls()); + addHostIDToDeleteHostsStatement = (ModificationStatement) QueryProcessor.getStatement(ADD_HOST_ID_TO_DELETE_HOSTS, ClientState + .forInternalCalls()); + setForceRepairStatement = (ModificationStatement) QueryProcessor.getStatement(SET_FORCE_REPAIR, ClientState + .forInternalCalls()); + clearDeleteHostsStatement = (ModificationStatement) QueryProcessor.getStatement(CLEAR_DELETE_HOSTS, ClientState + .forInternalCalls()); + delStatementRepairHistory = (ModificationStatement) QueryProcessor.getStatement(DEL_AUTO_REPAIR_HISTORY, ClientState + .forInternalCalls()); + Keyspace autoRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); + internalQueryCL = autoRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? + ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.ONE; + } + + public static class AutoRepairHistory + { + UUID hostId; + String repairTurn; + long lastRepairStartTime; + long lastRepairFinishTime; + Set deleteHosts; + long deleteHostsUpdateTime; + boolean forceRepair; + + public AutoRepairHistory(UUID hostId, String repairTurn, long lastRepairStartTime, long lastRepairFinishTime, + Set deleteHosts, long deleteHostsUpateTime, boolean forceRepair) + { + this.hostId = hostId; + this.repairTurn = repairTurn; + this.lastRepairStartTime = lastRepairStartTime; + this.lastRepairFinishTime = lastRepairFinishTime; + this.deleteHosts = deleteHosts; + if (this.deleteHosts == null) + { + this.deleteHosts = new HashSet<>(); + } + this.deleteHostsUpdateTime = deleteHostsUpateTime; + this.forceRepair = forceRepair; + } + + public String toString() + { + return MoreObjects.toStringHelper(this). + add("hostId", hostId). + add("repairTurn", repairTurn). + add("lastRepairStartTime", lastRepairStartTime). + add("lastRepairFinishTime", lastRepairFinishTime). + add("deleteHosts", deleteHosts). + toString(); + } + + public boolean isRepairRunning() + { + // if a repair history record has start time laster than finish time, it means the repair is running + return lastRepairStartTime > lastRepairFinishTime; + } + + public long getLastRepairFinishTime() + { + return lastRepairFinishTime; + } + } + + public static class CurrentRepairStatus + { + public Set hostIdsWithOnGoingRepair; // hosts that is running repair + public Set hostIdsWithOnGoingForceRepair; // hosts that is running repair because of force repair + Set priority; + public AutoRepairHistory myRepairHistory; + List historiesWithoutOnGoingRepair; // hosts that is NOT running repair + + public CurrentRepairStatus(List repairHistories, Set priority, UUID myId) + { + hostIdsWithOnGoingRepair = new HashSet<>(); + hostIdsWithOnGoingForceRepair = new HashSet<>(); + historiesWithoutOnGoingRepair = new ArrayList<>(); + + for (AutoRepairHistory history : repairHistories) + { + if (history.isRepairRunning()) + { + if (history.forceRepair) + { + hostIdsWithOnGoingForceRepair.add(history.hostId); + } + else + { + hostIdsWithOnGoingRepair.add(history.hostId); + } + } + else + { + historiesWithoutOnGoingRepair.add(history); + } + if (history.hostId.equals(myId)) + { + myRepairHistory = history; + } + } + this.priority = priority; + } + + public Set getAllHostsWithOngoingRepair() + { + return Sets.union(hostIdsWithOnGoingRepair, hostIdsWithOnGoingForceRepair); + } + + public String toString() + { + return MoreObjects.toStringHelper(this). + add("hostIdsWithOnGoingRepair", hostIdsWithOnGoingRepair). + add("hostIdsWithOnGoingForceRepair", hostIdsWithOnGoingForceRepair). + add("historiesWithoutOnGoingRepair", historiesWithoutOnGoingRepair). + add("priority", priority). + add("myRepairHistory", myRepairHistory). + toString(); + } + } + + @VisibleForTesting + public static List getAutoRepairHistory(RepairType repairType) + { + UntypedResultSet repairHistoryResult; + + ResultMessage.Rows repairStatusRows = selectStatementRepairHistory.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()))), Dispatcher.RequestTime.forImmediateExecution()); + repairHistoryResult = UntypedResultSet.create(repairStatusRows.result); + + List repairHistories = new ArrayList<>(); + if (!repairHistoryResult.isEmpty()) + { + for (UntypedResultSet.Row row : repairHistoryResult) + { + UUID hostId = row.getUUID(COL_HOST_ID); + String repairTurn = null; + if (row.has(COL_REPAIR_TURN)) + repairTurn = row.getString(COL_REPAIR_TURN); + long lastRepairStartTime = row.getLong(COL_REPAIR_START_TS, 0); + long lastRepairFinishTime = row.getLong(COL_REPAIR_FINISH_TS, 0); + Set deleteHosts = row.getSet(COL_DELETE_HOSTS, UUIDType.instance); + long deleteHostsUpdateTime = row.getLong(COL_DELETE_HOSTS_UPDATE_TIME, 0); + boolean forceRepair = row.has(COL_FORCE_REPAIR) && row.getBoolean(COL_FORCE_REPAIR); + repairHistories.add(new AutoRepairHistory(hostId, repairTurn, lastRepairStartTime, lastRepairFinishTime, + deleteHosts, deleteHostsUpdateTime, forceRepair)); + } + return repairHistories; + } + logger.info("No repair history found"); + return null; + } + + // A host may add itself in delete hosts for some other hosts due to restart or some temp gossip issue. If a node's record + // delete_hosts is not growing for more than 2 hours, we consider it as a normal node so we clear the delete_hosts for that node + public static void clearDeleteHosts(RepairType repairType, UUID hostId) + { + clearDeleteHostsStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId))), Dispatcher.RequestTime.forImmediateExecution()); + } + + public static void setForceRepairNewNode(RepairType repairType) + { + // this function will be called when a node bootstrap finished + UUID hostId = Gossiper.instance.getHostId(FBUtilities.getBroadcastAddressAndPort()); + // insert the data first + insertNewRepairHistory(repairType, currentTimeMillis(), currentTimeMillis()); + setForceRepair(repairType, hostId); + } + + public static void setForceRepair(RepairType repairType, Set hosts) + { + // this function is used by nodetool + for (InetAddressAndPort host : hosts) + { + UUID hostId = Gossiper.instance.getHostId(host); + setForceRepair(repairType, hostId); + } + } + + public static void setForceRepair(RepairType repairType, UUID hostId) + { + setForceRepairStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId))), + Dispatcher.RequestTime.forImmediateExecution()); + + logger.info("Set force repair repair type: {}, node: {}", repairType, hostId); + } + + public static long getLastRepairTimeForNode(RepairType repairType, UUID hostId) + { + ResultMessage.Rows rows = selectLastRepairTimeForNode.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList( + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId))), + Dispatcher.RequestTime.forImmediateExecution()); + UntypedResultSet repairTime = UntypedResultSet.create(rows.result); + if (repairTime.isEmpty()) + { + return 0; + } + return repairTime.one().getLong(COL_REPAIR_FINISH_TS); + } + + @VisibleForTesting + public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType, List autoRepairHistories, UUID myId) + { + if (autoRepairHistories != null) + { + return new CurrentRepairStatus(autoRepairHistories, getPriorityHostIds(repairType), myId); + } + return null; + } + + /** + * Checks whether the cluster has multiple major versions + * @return + * true if more than one major versions are detected + * false if only one major version is detected + * + */ + public static boolean hasMultipleLiveMajorVersions() + { + Set liveEndpoints = Gossiper.instance.getLiveMembers(); + Set majorVersions = new HashSet<>(); + for (InetAddressAndPort endpoint : liveEndpoints) + { + CassandraVersion releaseVersion = Gossiper.instance.getReleaseVersion(endpoint); + if (releaseVersion != null) + { + majorVersions.add(releaseVersion.major); + } + } + return majorVersions.size() > 1; + } + + /** + * Last version that does not support auto-repair. + * All nodes in the cluster must be running a version above this to enable auto-repair. + * Versions at or below this version (5.0.6) do not support auto-repair. + */ + @VisibleForTesting + static final CassandraVersion LAST_UNSUPPORTED_VERSION_FOR_AUTO_REPAIR = new CassandraVersion("5.0.6"); + + /** + * Checks whether any node in the cluster is running an unsupported version for auto-repair. + * + * @return true if any live node has a version at or below 5.0.6 (unsupported) or has an unknown version, + * false if all nodes are running versions above 5.0.6 (supported) + */ + public static boolean hasNodesBelowMinimumVersion() + { + Set liveEndpoints = Gossiper.instance.getLiveMembers(); + for (InetAddressAndPort endpoint : liveEndpoints) + { + CassandraVersion releaseVersion = Gossiper.instance.getReleaseVersion(endpoint); + if (releaseVersion == null) + { + logger.warn("Cannot determine version for endpoint {}, blocking auto-repair", endpoint); + return true; + } + if (releaseVersion.compareTo(LAST_UNSUPPORTED_VERSION_FOR_AUTO_REPAIR) <= 0) + { + logger.info("Endpoint {} is running version {} which does not support auto-repair " + + "(auto-repair requires version above {})", + endpoint, releaseVersion, LAST_UNSUPPORTED_VERSION_FOR_AUTO_REPAIR); + return true; + } + } + return false; + } + + @VisibleForTesting + protected static TreeSet getHostIdsInCurrentRing(RepairType repairType, Set allNodesInRing) + { + TreeSet hostIdsInCurrentRing = new TreeSet<>(); + for (InetAddressAndPort node : allNodesInRing) + { + String nodeDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(node); + if (AutoRepairService.instance.getAutoRepairConfig().getIgnoreDCs(repairType).contains(nodeDC)) + { + logger.info("Ignore node {} because its datacenter is {}", node, nodeDC); + continue; + } + /* + * Check if endpoint state exists in gossip or not. If it + * does not then this maybe a ghost node so ignore it + */ + if (Gossiper.instance.isAlive(node)) + { + UUID hostId = Gossiper.instance.getHostId(node); + hostIdsInCurrentRing.add(hostId); + } + else + { + logger.warn("Node is not present in Gossip cache node {}, node data center {}", node, nodeDC); + } + } + return hostIdsInCurrentRing; + } + + public static TreeSet getHostIdsInCurrentRing(RepairType repairType) + { + Set allNodesInRing = StorageService.instance.getTokenMetadata().getAllEndpoints(); + return getHostIdsInCurrentRing(repairType, allNodesInRing); + } + + // This function will return the host ID for the node which has not been repaired for longest time + public static AutoRepairHistory getHostWithLongestUnrepairTime(RepairType repairType) + { + List autoRepairHistories = getAutoRepairHistory(repairType); + return getHostWithLongestUnrepairTime(autoRepairHistories); + } + + /** + * Convenience method to resolve the broadcast address of a host id + * + * @return broadcast address if it exists, otherwise null. + */ + @Nullable + private static InetAddressAndPort getBroadcastAddress(UUID hostId) + { + return StorageService.instance.getEndpointForHostId(hostId); + } + + /** + * @return Map of broadcast address to host id, if a broadcast address cannot be found for a host, it is + * not included in the map. + */ + private static Map getBroadcastAddressToHostIdMap(Set hosts) + { + // Get a mapping of endpoint : host id + Map broadcastAddressMap = new HashMap<>(hosts.size()); + for (UUID hostId : hosts) + { + InetAddressAndPort broadcastAddress = getBroadcastAddress(hostId); + if (broadcastAddress == null) + { + logger.warn("Could not resolve broadcast address from host id {} in ClusterMetadata can't accurately " + + "determine if this node is a replica of the local node.", hostId); + } + else + { + broadcastAddressMap.put(broadcastAddress, hostId); + } + } + return broadcastAddressMap; + } + + /** + * @return Mapping of unique replication strategy to keyspaces using that strategy that we care about repairing. + */ + private static Map> getReplicationStrategies() + { + // Collect all unique replication strategies among all keyspaces. + Map> replicationStrategies = new HashMap<>(); + for (Keyspace keyspace : Keyspace.all()) + { + if (AutoRepairUtils.shouldConsiderKeyspace(keyspace)) + { + replicationStrategies.computeIfAbsent(keyspace.getReplicationStrategy(), k -> new ArrayList<>()) + .add(keyspace.getName()); + } + } + return replicationStrategies; + } + + /** + * Collects all hosts being repaired among all active repair schedules and their schedule if + * {@link AutoRepairConfig#getAllowParallelReplicaRepairAcrossSchedules(RepairType)} is true for this repairType. + * Accepts the currently evaluated repairType's schedule as an optimization to avoid grabbing its repair status an + * additional time. + * + * @param myRepairType The repair type schedule being evaluated. + * @param myRepairStatus The repair status for that repair type. + * @return All hosts among active schedules currently being repaired. + */ + private static Map getHostsBeingRepaired(RepairType myRepairType, CurrentRepairStatus myRepairStatus) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + + Map hostsBeingRepaired = myRepairStatus.getAllHostsWithOngoingRepair().stream() + .collect(Collectors.toMap((h) -> h, (v) -> myRepairType)); + + // If we don't allow repairing across schedules, iterate over other enabled schedules and include hosts + // actively being repaired. + if (!config.getAllowParallelReplicaRepairAcrossSchedules(myRepairType)) + { + for (RepairType repairType : RepairType.values()) + { + if (myRepairType == repairType) + continue; + + if (config.isAutoRepairEnabled(repairType)) + { + CurrentRepairStatus repairStatus = getCurrentRepairStatus(repairType, getAutoRepairHistory(repairType), null); + if (repairStatus != null) + { + for (UUID hostId : repairStatus.getAllHostsWithOngoingRepair()) + { + hostsBeingRepaired.putIfAbsent(hostId, repairType); + } + } + } + } + } + return hostsBeingRepaired; + } + + /** + * Identifies the most eligible host to repair for nodes preceding or equal to this nodes' lastRepairFinishTime. + * The criteria for this is to find the node with the oldest last repair finish time of which none of its replicas + * are currently under repair. + * + * @return The most eligible host to repair or null if no candidates before and including this nodes' current repair status. + */ + @VisibleForTesting + public static AutoRepairHistory getMostEligibleHostToRepair(RepairType repairType, CurrentRepairStatus currentRepairStatus, UUID myId) + { + // 0. If this repairType allows parallel replica repair, short circuit and return the host with the longest unrepair time + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + if (config.getAllowParallelReplicaRepair(repairType)) + { + return getHostWithLongestUnrepairTime(currentRepairStatus.historiesWithoutOnGoingRepair); + } + + // 1. Sort repair histories from oldest completed to newest + Stream finishedRepairHistories = currentRepairStatus.historiesWithoutOnGoingRepair + .stream() + .sorted(Comparator.comparingLong(h -> h.lastRepairFinishTime)); + + // 2. Optimization: Truncate repair histories after myId so we don't evaluate anything more recent as if we + // aren't interested in anything that isn't this node. + final AtomicBoolean myHistoryFound = new AtomicBoolean(false); + finishedRepairHistories = finishedRepairHistories + .filter(history -> { + if (myHistoryFound.get()) return false; // Stop processing after finding myId + myHistoryFound.set(history.hostId.equals(myId)); + return true; + }); + + // If there are any hosts with ongoing repair, filter the repair histories to not include nodes whose replicas + // are ongoing repair. + Map hostsBeingRepairedToRepairType = getHostsBeingRepaired(repairType, currentRepairStatus); + + // 3. If I am already actively being repaired in another schedule, defer submitting repairs; if already + // repairing for this type, return node so it can take its turn. + RepairType alreadyRepairingType = hostsBeingRepairedToRepairType.get(myId); + if (alreadyRepairingType != null) + { + if (repairType != alreadyRepairingType) + { + logger.info("Deferring repair because I am already actively repairing in schedule {}", hostsBeingRepairedToRepairType.get(myId)); + AutoRepairMetricsManager.getMetrics(repairType).repairDelayedBySchedule.inc(); + return null; + } + else if (currentRepairStatus.myRepairHistory != null) + { + // if the repair type matches this repair, assume the node was restarted while repairing, return node + // so it can take its turn. + logAlreadyMyTurn(); + return currentRepairStatus.myRepairHistory; + } + } + + if (!hostsBeingRepairedToRepairType.isEmpty()) + { + // 4. Extract InetAddresses for each UUID as replicas are identified by their address. + Map hostsBeingRepaired = getBroadcastAddressToHostIdMap(hostsBeingRepairedToRepairType.keySet()); + + // 5. Collect unique replication strategies and group them up with their keyspaces. + Map> replicationStrategies = getReplicationStrategies(); + + // 6. Filter out repair histories who have a replica being repaired, note that this is lazy, given the stream + // is completed using findFirst, it should stop as soon as the matching criteria is met. + finishedRepairHistories = finishedRepairHistories.filter((history) -> !hasReplicaWithOngoingRepair(history, + myId, + repairType, + hostsBeingRepaired, + hostsBeingRepairedToRepairType, + replicationStrategies)); + } + + // 7. Select the first (oldest lastRepairFinishTime) repair history without replicas being repaired + return finishedRepairHistories.findFirst().orElse(null); + } + + + /** + * @param eligibleHistory History of node to check + * @param myId Host id of this node, if the repair history is for this node, additional logging will take place. + * @param myRepairType repair type being evaluated + * @param hostsBeingRepaired Hosts being repaired. + * @param hostIdToRepairType mapping of hosts being repaired to the repair type its being repaired for. + * @param replicationStrategies Mapping of unique replication strategies to keyspaces having that strategy. + * @return Whether the host for the given eligibleRepairHistory has any replicas in hostsBeingRepaired. + */ + private static boolean hasReplicaWithOngoingRepair(AutoRepairHistory eligibleHistory, + UUID myId, + RepairType myRepairType, + Map hostsBeingRepaired, + Map hostIdToRepairType, + Map> replicationStrategies) + { + // If no broadcast address found for this host id in cluster metadata, just skip it, a node should always + // see itself in cluster metadata. + InetAddressAndPort eligibleBroadcastAddress = getBroadcastAddress(eligibleHistory.hostId); + if (eligibleBroadcastAddress == null) + { + return true; + } + + // For each replication strategy, determine if host being repaired is a replica of the local node. + for (Map.Entry> entry : replicationStrategies.entrySet()) + { + AbstractReplicationStrategy replicationStrategy = entry.getKey(); + EndpointsByRange endpointsByRange = replicationStrategy.getRangeAddresses(StorageService.instance.getTokenMetadata()); + + // get ranges of the eligible address for the given replication strategy. + RangesAtEndpoint rangesAtEndpoint = StorageService.instance.getReplicas(replicationStrategy, eligibleBroadcastAddress); + for (Replica replica : rangesAtEndpoint) + { + // get the endpoints involved in this range. + EndpointsForRange endpointsForRange = endpointsByRange.get(replica.range()); + // For each host in this range... + for (InetAddressAndPort inetAddressAndPort : endpointsForRange.endpoints()) + { + // If the address of the node in the range belongs to a host being repaired, return true. + UUID hostId = hostsBeingRepaired.get(inetAddressAndPort); + if (hostId != null) + { + // log if the repair history matches the current running node. + InetAddressAndPort myBroadcastAddress = getBroadcastAddress(myId); + if (myBroadcastAddress != null && myBroadcastAddress.equals(eligibleBroadcastAddress)) + { + logger.info("Deferring repair because replica {} ({}) with shared ranges for " + + "{} keyspace(s) (e.g. {}) is currently taking its turn for schedule {}", + hostId, inetAddressAndPort, entry.getValue().size(), entry.getValue().get(0), + hostIdToRepairType.get(hostId)); + AutoRepairMetricsManager.getMetrics(myRepairType).repairDelayedByReplica.inc(); + } + else if (logger.isDebugEnabled()) + { + logger.debug("Not considering node {} ({}) for repair as it has replica {} ({}) with " + + "shared ranges for {} keyspace(s) (e.g. {}) which is currently taking its " + + "turn for schedule {}", + eligibleHistory.hostId, eligibleBroadcastAddress, + hostId, inetAddressAndPort, entry.getValue().size(), entry.getValue().get(0), + hostIdToRepairType.get(hostId)); + } + return true; + } + } + } + } + + // No replicas found of eligible host. + return false; + } + + private static AutoRepairHistory getHostWithLongestUnrepairTime(List autoRepairHistories) + { + if (autoRepairHistories == null) + { + return null; + } + AutoRepairHistory rst = null; + long oldestTimestamp = Long.MAX_VALUE; + for (AutoRepairHistory autoRepairHistory : autoRepairHistories) + { + if (autoRepairHistory.lastRepairFinishTime < oldestTimestamp) + { + rst = autoRepairHistory; + oldestTimestamp = autoRepairHistory.lastRepairFinishTime; + } + } + return rst; + } + + public static int getMaxNumberOfNodeRunAutoRepair(RepairType repairType, int groupSize) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + if (groupSize == 0) + { + return Math.max(config.getParallelRepairCount(repairType), 1); + } + // we will use the max number from config between auto_repair_parallel_repair_count_in_group and auto_repair_parallel_repair_percentage_in_group + int value = Math.max(groupSize * config.getParallelRepairPercentage(repairType) / 100, + config.getParallelRepairCount(repairType)); + // make sure at least one node getting repaired + return Math.max(1, value); + } + + private static void logAlreadyMyTurn() + { + logger.warn("This node already was considered to having an ongoing repair for this repair type, must have " + + "been restarted, taking my turn back"); + } + + @VisibleForTesting + public static RepairTurn myTurnToRunRepair(RepairType repairType, UUID myId) + { + try + { + Set allNodesInRing = StorageService.instance.getTokenMetadata().getAllEndpoints(); + logger.info("Total nodes in ring {}", allNodesInRing.size()); + TreeSet hostIdsInCurrentRing = getHostIdsInCurrentRing(repairType, allNodesInRing); + logger.info("Total nodes qualified for repair {}", hostIdsInCurrentRing.size()); + + List autoRepairHistories = getAutoRepairHistory(repairType); + Set autoRepairHistoryIds = new HashSet<>(); + + // 1. Remove any node that is not part of group based on gossip info + if (autoRepairHistories != null) + { + for (AutoRepairHistory nodeHistory : autoRepairHistories) + { + autoRepairHistoryIds.add(nodeHistory.hostId); + // clear delete_hosts if the node's delete hosts is not growing for more than two hours + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + if (!nodeHistory.deleteHosts.isEmpty() + && config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds() < TimeUnit.MILLISECONDS.toSeconds( + currentTimeMillis() - nodeHistory.deleteHostsUpdateTime + )) + { + clearDeleteHosts(repairType, nodeHistory.hostId); + logger.info("Delete hosts for {} for repair type {} has not been updated for more than {} seconds. Delete hosts has been cleared. Delete hosts before clear {}" + , nodeHistory.hostId, repairType, config.getAutoRepairHistoryClearDeleteHostsBufferInterval(), nodeHistory.deleteHosts); + } + else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) + { + if (nodeHistory.deleteHosts.size() > Math.max(2, hostIdsInCurrentRing.size() * 0.5)) + { + // More than half of the groups thinks the record should be deleted + logger.info("{} think {} is orphan node, will delete auto repair history for repair type {}.", nodeHistory.deleteHosts, nodeHistory.hostId, repairType); + deleteAutoRepairHistory(repairType, nodeHistory.hostId); + } + else + { + // I think this host should be deleted + logger.info("I({}) think {} is not part of ring, vote to delete it for repair type {}.", myId, nodeHistory.hostId, repairType); + addHostIdToDeleteHosts(repairType, myId, nodeHistory.hostId); + } + } + } + } + + // 2. Add node to auto repair history table if a node is in gossip info + for (UUID hostId : hostIdsInCurrentRing) + { + if (!autoRepairHistoryIds.contains(hostId)) + { + logger.info("{} for repair type {} doesn't exist in the auto repair history table, insert a new record.", repairType, hostId); + insertNewRepairHistory(repairType, hostId, currentTimeMillis(), currentTimeMillis()); + } + } + + // get updated current repair status + CurrentRepairStatus currentRepairStatus = getCurrentRepairStatus(repairType, getAutoRepairHistory(repairType), myId); + if (currentRepairStatus != null) + { + if (logger.isDebugEnabled()) + { + logger.debug("Latest repair status {}", currentRepairStatus); + } + //check if I am forced to run repair + for (AutoRepairHistory history : currentRepairStatus.historiesWithoutOnGoingRepair) + { + if (history.forceRepair && history.hostId.equals(myId)) + { + return MY_TURN_FORCE_REPAIR; + } + } + } + + // check if node was already indicated as having an ongoing repair, this may happen when a node restarts + // before finishing repairing. + if (currentRepairStatus != null && currentRepairStatus.getAllHostsWithOngoingRepair().contains(myId)) + { + logAlreadyMyTurn(); + + // use the previously chosen turn. + if (currentRepairStatus.myRepairHistory != null && currentRepairStatus.myRepairHistory.repairTurn != null) + { + return RepairTurn.valueOf(currentRepairStatus.myRepairHistory.repairTurn); + } + else + { + return MY_TURN; + } + } + + int parallelRepairNumber = getMaxNumberOfNodeRunAutoRepair(repairType, + autoRepairHistories == null ? 0 : autoRepairHistories.size()); + logger.info("Will run repairs concurrently on {} node(s)", parallelRepairNumber); + if (currentRepairStatus == null || parallelRepairNumber > currentRepairStatus.hostIdsWithOnGoingRepair.size()) + { + // more repairs can be run, I might be the new one + if (autoRepairHistories != null) + { + logger.info("Auto repair history table has {} records", autoRepairHistories.size()); + } + else + { + // try to fetch again + autoRepairHistories = getAutoRepairHistory(repairType); + if (autoRepairHistories == null) + { + logger.error("No record found"); + return NOT_MY_TURN; + } + + currentRepairStatus = getCurrentRepairStatus(repairType, autoRepairHistories, myId); + } + + UUID priorityHostId = null; + if (currentRepairStatus.priority != null) + { + for (UUID priorityID : currentRepairStatus.priority) + { + // remove ids doesn't belong to this ring + if (!hostIdsInCurrentRing.contains(priorityID)) + { + logger.info("{} is not part of the current ring, will be removed from priority list.", priorityID); + removePriorityStatus(repairType, priorityID); + } + else + { + priorityHostId = priorityID; + break; + } + } + } + + if (priorityHostId != null && !myId.equals(priorityHostId)) + { + logger.info("Priority list is not empty and I'm not the first node in the list, not my turn." + + "First node in priority list is {}", getBroadcastAddress(priorityHostId)); + return NOT_MY_TURN; + } + + if (myId.equals(priorityHostId)) + { + //I have a priority for repair hence its my turn now + return MY_TURN_DUE_TO_PRIORITY; + } + + // Determine if this node is the most eligible host to repair. + AutoRepairHistory nodeToBeRepaired = getMostEligibleHostToRepair(repairType, currentRepairStatus, myId); + if (nodeToBeRepaired != null) + { + if (nodeToBeRepaired.hostId.equals(myId)) + { + logger.info("This node is selected to be repaired for repair type {}", repairType); + return MY_TURN; + } + + // log which node is next, which is helpful for debugging + logger.info("Next node to be repaired for repair type {}: {} ({})", repairType, + getBroadcastAddress(nodeToBeRepaired.hostId), + nodeToBeRepaired); + } + + // If this node is not identified as most eligible, set the repair lag time. + if (currentRepairStatus.myRepairHistory != null) + { + AutoRepairMetricsManager.getMetrics(repairType) + .recordRepairStartLag(currentRepairStatus.myRepairHistory.lastRepairFinishTime); + } + } + else if (currentRepairStatus.hostIdsWithOnGoingForceRepair.contains(myId)) + { + return MY_TURN_FORCE_REPAIR; + } + // for some reason I was not done with the repair hence resume (maybe node restart in-between, etc.) + return currentRepairStatus.hostIdsWithOnGoingRepair.contains(myId) ? MY_TURN : NOT_MY_TURN; + } + catch (Exception e) + { + logger.error("Exception while deciding node's turn:", e); + } + return NOT_MY_TURN; + } + + static void deleteAutoRepairHistory(RepairType repairType, UUID hostId) + { + //delete the given hostId + delStatementRepairHistory.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId))), Dispatcher.RequestTime.forImmediateExecution()); + } + + static void updateStartAutoRepairHistory(RepairType repairType, UUID myId, long timestamp, RepairTurn turn) + { + recordStartRepairHistoryStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(timestamp), + ByteBufferUtil.bytes(turn.name()), + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(myId) + )), Dispatcher.RequestTime.forImmediateExecution()); + } + + static void updateFinishAutoRepairHistory(RepairType repairType, UUID myId, long timestamp) + { + recordFinishRepairHistoryStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(timestamp), + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(myId) + )), Dispatcher.RequestTime.forImmediateExecution()); + logger.info("Auto repair finished for {}", myId); + } + + public static void insertNewRepairHistory(RepairType repairType, UUID hostId, long startTime, long finishTime) + { + try + { + Keyspace autoRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); + ConsistencyLevel cl = autoRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? + ConsistencyLevel.LOCAL_SERIAL : null; + + UntypedResultSet resultSet; + ResultMessage.Rows resultMessage = (ResultMessage.Rows) insertNewRepairHistoryStatement.execute( + QueryState.forInternalCalls(), QueryOptions.create(internalQueryCL, Lists.newArrayList( + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId), + ByteBufferUtil.bytes(startTime), + ByteBufferUtil.bytes(finishTime), + ByteBufferUtil.bytes(currentTimeMillis()) + ), false, -1, null, cl, ProtocolVersion.CURRENT, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME), + Dispatcher.RequestTime.forImmediateExecution()); + resultSet = UntypedResultSet.create(resultMessage.result); + boolean applied = resultSet.one().getBoolean(ModificationStatement.CAS_RESULT_COLUMN.toString()); + if (applied) + { + logger.info("Successfully inserted a new auto repair history record for host id: {}", hostId); + } + else + { + logger.info("Record exists, no need to insert again for host id: {}", hostId); + } + } + catch (Exception e) + { + logger.error("Exception in inserting new repair history:", e); + } + } + + public static void insertNewRepairHistory(RepairType repairType, long startTime, long finishTime) + { + UUID hostId = Gossiper.instance.getHostId(FBUtilities.getBroadcastAddressAndPort()); + insertNewRepairHistory(repairType, hostId, startTime, finishTime); + } + + public static void addHostIdToDeleteHosts(RepairType repairType, UUID myID, UUID hostToBeDeleted) + { + SetSerializer serializer = SetSerializer.getInstance(UUIDSerializer.instance, UTF8Type.instance.comparatorSet); + addHostIDToDeleteHostsStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(serializer.serialize(new HashSet<>(Arrays.asList(myID))), + ByteBufferUtil.bytes(currentTimeMillis()), + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostToBeDeleted) + )), Dispatcher.RequestTime.forImmediateExecution()); + } + + public static void addPriorityHosts(RepairType repairType, Set hosts) + { + Set hostIds = new HashSet<>(); + for (InetAddressAndPort host : hosts) + { + //find hostId from IP address + UUID hostId = StorageService.instance.getTokenMetadata().getHostId(host); + hostIds.add(hostId); + if (hostId != null) + { + logger.info("Add host {} to the priority list", hostId); + } + } + if (!hostIds.isEmpty()) + { + SetSerializer serializer = SetSerializer.getInstance(UUIDSerializer.instance, UTF8Type.instance.comparatorSet); + addPriorityHost.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(serializer.serialize(hostIds), + ByteBufferUtil.bytes(repairType.toString()))), + Dispatcher.RequestTime.forImmediateExecution()); + } + } + + static void removePriorityStatus(RepairType repairType, UUID hostId) + { + logger.info("Remove host {} from priority list", hostId); + delStatementPriorityStatus.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(hostId), + ByteBufferUtil.bytes(repairType.toString()))), + Dispatcher.RequestTime.forImmediateExecution()); + } + + public static Set getPriorityHostIds(RepairType repairType) + { + UntypedResultSet repairPriorityResult; + + ResultMessage.Rows repairPriorityRows = selectStatementRepairPriority.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()))), Dispatcher.RequestTime.forImmediateExecution()); + repairPriorityResult = UntypedResultSet.create(repairPriorityRows.result); + + Set priorities = null; + if (!repairPriorityResult.isEmpty()) + { + // there should be only one row + UntypedResultSet.Row row = repairPriorityResult.one(); + priorities = row.getSet(COL_REPAIR_PRIORITY, UUIDType.instance); + } + if (priorities != null) + { + return priorities; + } + return Collections.emptySet(); + } + + public static Set getPriorityHosts(RepairType repairType) + { + Set hosts = new HashSet<>(); + for (UUID hostId : getPriorityHostIds(repairType)) + { + InetAddressAndPort broadcastAddress = getBroadcastAddress(hostId); + if (broadcastAddress == null) + { + logger.warn("Could not resolve broadcastAddress for {}, skipping considering it as a priority host", hostId); + continue; + } + hosts.add(broadcastAddress); + } + return hosts; + } + + public static boolean shouldConsiderKeyspace(Keyspace ks) + { + AbstractReplicationStrategy replicationStrategy = ks.getReplicationStrategy(); + boolean repair = true; + if (replicationStrategy instanceof NetworkTopologyStrategy) + { + Set datacenters = ((NetworkTopologyStrategy) replicationStrategy).getDatacenters(); + String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort()); + if (!datacenters.contains(localDC)) + { + repair = false; + } + } + if (replicationStrategy instanceof LocalStrategy) + { + repair = false; + } + if (ks.getName().equalsIgnoreCase(SchemaConstants.TRACE_KEYSPACE_NAME)) + { + // by default, ignore the tables under system_traces as they do not have + // that much important data + repair = false; + } + return repair; + } + + public static boolean tableMaxRepairTimeExceeded(RepairType repairType, long startTime) + { + long tableRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds + (currentTimeMillis() - startTime); + return AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() < + tableRepairTimeSoFar; + } + + public static boolean keyspaceMaxRepairTimeExceeded(RepairType repairType, long startTime, int numOfTablesToBeRepaired) + { + long keyspaceRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds((currentTimeMillis() - startTime)); + return (long) AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() * + numOfTablesToBeRepaired < keyspaceRepairTimeSoFar; + } + + public static List getAllMVs(RepairType repairType, Keyspace keyspace, TableMetadata tableMetadata) + { + List allMvs = new ArrayList<>(); + if (AutoRepairService.instance.getAutoRepairConfig().getMaterializedViewRepairEnabled(repairType) && keyspace.getMetadata().views != null) + { + Iterator views = keyspace.getMetadata().views.forTable(tableMetadata.id).iterator(); + while (views.hasNext()) + { + String viewName = views.next().name(); + logger.info("Adding MV to the list {}.{}.{}", keyspace.getName(), tableMetadata.name, viewName); + allMvs.add(viewName); + } + } + return allMvs; + } + + public static Collection> split(Range tokenRange, int numberOfSplits) + { + Collection> ranges; + Optional splitter = DatabaseDescriptor.getPartitioner().splitter(); + if (!splitter.isPresent()) + { + NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 30, TimeUnit.MINUTES, "Partitioner {} does not support splitting, falling back to splitting by token range", DatabaseDescriptor.getPartitioner()); + ranges = Collections.singleton(tokenRange); + } + else + { + ranges = splitter.get().split(Collections.singleton(tokenRange), numberOfSplits); + } + return ranges; + } + + /** + * Finds a list of SSTables for a given {@code repairType}, + * {@code keyspace}, {@code table}, and {@code tokenRange} and then it internally calls + * another API {@code AutoRepairUtils.getSizesForRangeOfSSTables}, which figures out the estimated data size. + * + * @param repairType the repair type (e.g., FULL, INCREMENTAL) + * @param keyspace the keyspace name + * @param table the table name + * @param tokenRange the token range to evaluate + * @return an estimate representing the number of partitions, size in range, and total size + */ + static SizeEstimate getRangeSizeEstimate(RepairType repairType, String keyspace, String table, Range tokenRange) + { + logger.debug("Calculating size estimate for {}.{} for range {}", keyspace, table, tokenRange); + try (Refs refs = RepairTokenRangeSplitter.getSSTableReaderRefs(repairType, keyspace, table, tokenRange)) + { + SizeEstimate estimate = getSizesForRangeOfSSTables(repairType, keyspace, table, tokenRange, refs); + logger.debug("Generated size estimate {}", estimate); + return estimate; + } + } + /** + * Calculates the size estimation qualified to be repaired for a given {@code repairType}, + * {@code keyspace}, {@code table}, {@code tokenRange}, and {@code refs}. + *

+ * If the compression is enabled, then the size will be an estimate, otherwise it will be accurate. + *

+ * + * @param repairType + * @param keyspace + * @param table + * @param tokenRange + * @param refs + * @return an estimate representing the number of partitions, size in range, and total size + */ + static SizeEstimate getSizesForRangeOfSSTables(RepairType repairType, String keyspace, String table, + Range tokenRange, Refs refs) + { + List> singletonRange = Collections.singletonList(tokenRange); + ICardinality cardinality = new HyperLogLogPlus(13, 25); + long approxBytesInRange = 0L; + long totalBytes = 0L; + + for (SSTableReader reader : refs) + { + try + { + if (reader.openReason == SSTableReader.OpenReason.EARLY) + continue; + CompactionMetadata metadata = (CompactionMetadata) reader.descriptor.getMetadataSerializer().deserialize(reader.descriptor, MetadataType.COMPACTION); + if (metadata != null) + cardinality = cardinality.merge(metadata.cardinalityEstimator); + + // use onDiskLength, which is the actual size of the SSTable data file. + long sstableSize = reader.onDiskLength(); + totalBytes += sstableSize; + + // get the on disk size for the token range, note for compressed data this includes the full + // chunks the start and end ranges are found in. + long approximateRangeBytesInSSTable = reader.onDiskSizeForPartitionPositions(reader.getPositionsForRanges(singletonRange)); + approxBytesInRange += Math.min(approximateRangeBytesInSSTable, sstableSize); + } + catch (IOException | CardinalityMergeException e) + { + logger.error("Error calculating size estimate for {}.{} for range {} on {}", keyspace, table, tokenRange, reader, e); + } + } + + long partitions = 0L; + if (totalBytes > 0) + { + // use the ratio from size to estimate the partitions in the range as well + double ratio = approxBytesInRange / (double) totalBytes; + partitions = (long) Math.max(1, Math.ceil(cardinality.cardinality() * ratio)); + } + return new SizeEstimate(repairType, keyspace, table, tokenRange, partitions, approxBytesInRange, totalBytes); + } + + /** + * Calculates the token ranges owned by this node for a given keyspace. + * + * @param primaryRangeOnly whether to use only primary token ranges or include replicated ones + * @param keyspaceName the name of the keyspace + * @return one or more token ranges owned by this node + */ + public static List> getTokenRanges(boolean primaryRangeOnly, String keyspaceName) + { + // Collect all applicable token ranges + Collection> wrappedRanges; + if (primaryRangeOnly) + { + wrappedRanges = StorageService.instance.getPrimaryRanges(keyspaceName); + } + else + { + wrappedRanges = StorageService.instance.getLocalRanges(keyspaceName); + } + + // Unwrap each range as we need to account for ranges that overlap the ring + List> ranges = new ArrayList<>(); + for (Range wrappedRange : wrappedRanges) + { + ranges.addAll(wrappedRange.unwrap()); + } + return ranges; + } + + /** + * Calculates the total bytes to be repaired for a given keyspace and list of tables. + * + * @param repairType the repair type (e.g., FULL, INCREMENTAL) + * @param keyspaceName the name of the keyspace + * @param tableNames the list of tables + * @return a key-value map where the key is {@code keyspaceName.tableName} and the value is the number of bytes + * to be repaired. + */ + public static Map, SizeEstimate>> calcTotalBytesToBeRepaired(RepairType repairType, String keyspaceName, List tableNames, List> tokenRanges) + { + Map, SizeEstimate>> ksTablesEstimatedBytes = new HashMap<>(); + for (String tableName : tableNames) + { + String ksTable = getKeyspaceTableName(keyspaceName, tableName); + ksTablesEstimatedBytes.computeIfAbsent(ksTable, k -> new HashMap<>()); + Map, SizeEstimate> tokenToSize = ksTablesEstimatedBytes.get(ksTable); + for (Range tokenRange : tokenRanges) + { + SizeEstimate tableAssignments = getRangeSizeEstimate(repairType, keyspaceName, tableName, tokenRange); + tokenToSize.put(tokenRange, tableAssignments); + } + } + return ksTablesEstimatedBytes; + } + + public static String getKeyspaceTableName(String keyspace, String table) + { + return keyspace + "." + table; + } + + /** + * Represents a size estimate by both bytes and partition count for a given keyspace and table for a token range. + */ + @VisibleForTesting + protected static class SizeEstimate + { + public final RepairType repairType; + public final String keyspace; + public final String table; + public final Range tokenRange; + public final long partitions; + public final long sizeInRange; + public final long totalSize; + /** + * Size to consider in the repair. For incremental repair, we want to consider the total size + * of the estimate as we have to factor in anticompacting the entire SSTable. + * For full repair, just use the size containing the range. + */ + public final long sizeForRepair; + + public SizeEstimate(RepairType repairType, + String keyspace, String table, Range tokenRange, + long partitions, long sizeInRange, long totalSize) + { + this.repairType = repairType; + this.keyspace = keyspace; + this.table = table; + this.tokenRange = tokenRange; + this.partitions = partitions; + this.sizeInRange = sizeInRange; + this.totalSize = totalSize; + + this.sizeForRepair = repairType == RepairType.INCREMENTAL ? totalSize : sizeInRange; + } + + @Override + public String toString() + { + return "SizeEstimate{" + + "repairType=" + repairType + + ", keyspace='" + keyspace + '\'' + + ", table='" + table + '\'' + + ", tokenRange=" + tokenRange + + ", partitions=" + partitions + + ", sizeInRange=" + sizeInRange + + ", totalSize=" + totalSize + + ", sizeForRepair=" + sizeForRepair + + '}'; + } + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitter.java new file mode 100644 index 000000000000..1b2c80b74271 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitter.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.repair.autorepair; + + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.service.AutoRepairService; + +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; + +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.split; + +/** + * An implementation that splits token ranges into a fixed number of subranges. + */ +public class FixedSplitTokenRangeSplitter implements IAutoRepairTokenRangeSplitter +{ + private static final Logger logger = LoggerFactory.getLogger(FixedSplitTokenRangeSplitter.class); + + /** + * Selecting the default value is tricky. If we select a small number, individual repairs would be heavy. + * On the other hand, if we select a large number, too many repair sessions would be created. + *

+ * If vnodes are configured using num_tokens, attempts to evenly subdivide subranges by each range + * using the following formula: + *

+ * Math.max(1, numberOfSubranges / tokens.size()) + *

+ * To maintain balance, 32 serves as a good default that accommodates both vnodes and non-vnodes effectively. + */ + public static final int DEFAULT_NUMBER_OF_SUBRANGES = 32; + + /** + * Number of evenly split subranges to create for each node that repair runs for. + *

+ * If vnodes are configured using num_tokens, attempts to evenly subdivide subranges by each range. + * For example, for num_tokens: 16 and number_of_subranges: 32, 2 (32/16) + * repair assignments will be created for each token range. At least one repair assignment will be + * created for each token range. + */ + static final String NUMBER_OF_SUBRANGES = "number_of_subranges"; + + private final AutoRepairConfig.RepairType repairType; + private int numberOfSubranges; + + public FixedSplitTokenRangeSplitter(AutoRepairConfig.RepairType repairType, Map parameters) + { + this.repairType = repairType; + + numberOfSubranges = Integer.parseInt(parameters.getOrDefault(NUMBER_OF_SUBRANGES, Integer.toString(DEFAULT_NUMBER_OF_SUBRANGES))); + } + + @Override + public Iterator getRepairAssignments(boolean primaryRangeOnly, List repairPlans) + { + return new RepairAssignmentIterator(repairPlans) + { + @Override + protected KeyspaceRepairAssignments next(int priority, KeyspaceRepairPlan repairPlan) + { + return getRepairAssignmentsForKeyspace(primaryRangeOnly, priority, repairPlan); + } + }; + } + + private KeyspaceRepairAssignments getRepairAssignmentsForKeyspace(boolean primaryRangeOnly, int priority, KeyspaceRepairPlan repairPlan) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + List repairAssignments = new ArrayList<>(); + String keyspaceName = repairPlan.getKeyspaceName(); + List tableNames = repairPlan.getTableNames(); + + Collection> tokens = AutoRepairUtils.getTokenRanges(primaryRangeOnly, keyspaceName); + boolean byKeyspace = config.getRepairByKeyspace(repairType); + // collect all token ranges. + List> allRanges = new ArrayList<>(); + // this is done to avoid micro splits in the case of vnodes + int splitsPerRange = Math.max(1, numberOfSubranges / tokens.size()); + for (Range token : tokens) + { + allRanges.addAll(split(token, splitsPerRange)); + } + + if (byKeyspace) + { + + // This calculation is the best effort for the FixedSplitTokenRangeSplitter. + // In practice, this metric may not give you an accurate view in case of uneven data distribution. + long totalBytes = repairPlan.getEstimatedBytes(); + long bytesPerRange = Math.max(1, totalBytes / splitsPerRange); + for (Range splitRange : allRanges) + { + // add repair assignment for each range entire keyspace's tables + repairAssignments.add(new RepairAssignment(splitRange, keyspaceName, tableNames, bytesPerRange)); + } + } + else + { + // add repair assignment per table + for (String tableName : tableNames) + { + long totalBytes = repairPlan.getTableEstimatedBytes(AutoRepairUtils.getKeyspaceTableName(keyspaceName, tableName)); + long bytesPerRange = Math.max(1, totalBytes / splitsPerRange); + for (Range splitRange : allRanges) + { + repairAssignments.add(new RepairAssignment(splitRange, keyspaceName, Collections.singletonList(tableName), bytesPerRange)); + } + } + } + return new KeyspaceRepairAssignments(priority, keyspaceName, repairAssignments); + } + + @Override + public void setParameter(String key, String value) + { + if (!key.equals(NUMBER_OF_SUBRANGES)) + { + throw new IllegalArgumentException("Unexpected parameter '" + key + "', must be " + NUMBER_OF_SUBRANGES); + } + logger.info("Setting {} to {} for repair type {}", key, value, repairType); + this.numberOfSubranges = Integer.parseInt(value); + } + + @Override + public Map getParameters() + { + return Collections.singletonMap(NUMBER_OF_SUBRANGES, Integer.toString(numberOfSubranges)); + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java new file mode 100644 index 000000000000..8b82eac296db --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.repair.autorepair; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.cassandra.config.ParameterizedClass; + +/** + * Interface that defines how to generate {@link KeyspaceRepairAssignments}. + *

+ * The default is {@link RepairTokenRangeSplitter} which aims to provide sensible defaults for all repair types. + *

+ * Custom implementations class should require a constructor accepting + * ({@link AutoRepairConfig.RepairType}, {@link java.util.Map}) with the {@link java.util.Map} parameter accepting + * custom configuration for your splitter. If such a constructor does not exist, + * {@link AutoRepairConfig#newAutoRepairTokenRangeSplitter(AutoRepairConfig.RepairType, ParameterizedClass)} + * will fall back on invoking a default zero argument constructor. + */ +public interface IAutoRepairTokenRangeSplitter +{ + /** + * Split the token range you wish to repair into multiple assignments. + * The autorepair framework will repair the assignments from returned subrange iterator in the sequence it's + * provided. + * @param primaryRangeOnly Whether to repair only this node's primary ranges or all of its ranges. + * @param repairPlans A list of ordered prioritized repair plans to generate assignments for in order. + * @return iterator of repair assignments, with each element representing a grouping of repair assignments for a given keyspace. + * The iterator is traversed lazily {@link KeyspaceRepairAssignments} at a time with the intent to try to get the + * most up-to-date representation of your data (e.g. how much data exists and is unrepaired at a given time). + */ + Iterator getRepairAssignments(boolean primaryRangeOnly, List repairPlans); + + /** + * Update a configuration parameter. This is meant to be used by nodetool setautorepairconfig to + * update configuration dynamically. + * @param key parameter to update + * @param value The value to set to. + */ + default void setParameter(String key, String value) + { + throw new IllegalArgumentException(this.getClass().getName() + " does not support custom configuration"); + } + + /** + * @return custom configuration. This is meant to be used by nodetool getautorepairconfig for + * retrieving the splitter configuration. + */ + default Map getParameters() + { + return Collections.emptyMap(); + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairAssignments.java b/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairAssignments.java new file mode 100644 index 000000000000..3ea91e9922f9 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairAssignments.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.List; + +/** + * A grouping of repair assignments that were generated for a particular keyspace for a given priority. + */ +public class KeyspaceRepairAssignments +{ + private final int priority; + private final String keyspaceName; + private final List repairAssignments; + + public KeyspaceRepairAssignments(int priority, String keyspaceName, List repairAssignments) + { + this.priority = priority; + this.keyspaceName = keyspaceName; + this.repairAssignments = repairAssignments; + } + + public int getPriority() + { + return priority; + } + + public String getKeyspaceName() + { + return keyspaceName; + } + + public List getRepairAssignments() + { + return repairAssignments; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairPlan.java b/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairPlan.java new file mode 100644 index 000000000000..750e56e9d2d3 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairPlan.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; + +/** + * Encapsulates an intent to repair the given keyspace's tables + */ +public class KeyspaceRepairPlan +{ + private final String keyspaceName; + + private final List tableNames; + + @VisibleForTesting + public Map, AutoRepairUtils.SizeEstimate>> ksTablesEstimatedBytes; + + public KeyspaceRepairPlan(String keyspaceName, List tableNames, Map, AutoRepairUtils.SizeEstimate>> ksTablesEstimatedBytes) + { + this.keyspaceName = keyspaceName; + this.tableNames = tableNames; + this.ksTablesEstimatedBytes = ksTablesEstimatedBytes; + } + + public String getKeyspaceName() + { + return keyspaceName; + } + + public List getTableNames() + { + return tableNames; + } + + public long getEstimatedBytes() + { + return ksTablesEstimatedBytes.values().stream() + .flatMap(tableMap -> tableMap.values().stream()) + .mapToLong(sizeEstimate -> sizeEstimate.sizeForRepair) + .sum(); + } + + public long getTableEstimatedBytes(String keyspaceTableName) + { + return ksTablesEstimatedBytes.getOrDefault(keyspaceTableName, + Collections.emptyMap()).values().stream().mapToLong(sizeEstimate -> sizeEstimate.sizeForRepair).sum(); + } + + public AutoRepairUtils.SizeEstimate getSizeEstimate(String keyspaceTableName, Range tokenRange) + { + return ksTablesEstimatedBytes == null ? null + : ksTablesEstimatedBytes.getOrDefault(keyspaceTableName, null) == null ? null + : ksTablesEstimatedBytes.get(keyspaceTableName).get(tokenRange); + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + KeyspaceRepairPlan that = (KeyspaceRepairPlan) o; + return Objects.equals(keyspaceName, that.keyspaceName) && Objects.equals(tableNames, that.tableNames) + && Objects.equals(ksTablesEstimatedBytes, that.ksTablesEstimatedBytes); + } + + @Override + public int hashCode() + { + return Objects.hash(keyspaceName, tableNames, ksTablesEstimatedBytes); + } + + @Override + public String toString() + { + return "KeyspaceRepairPlan{" + + "keyspaceName='" + keyspaceName + '\'' + + ", tableNames=" + tableNames + + ", ksTablesEstimatedBytes=" + ksTablesEstimatedBytes + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java b/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java new file mode 100644 index 000000000000..4457ccdd348e --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeSet; +import java.util.function.Consumer; + +import org.apache.cassandra.db.ColumnFamilyStore; + +/** + * Encapsulates a devised plan to repair tables, grouped by their keyspace and a given priority. This is used + * by {@link AutoRepair} to pass in an organized plan to + * {@link IAutoRepairTokenRangeSplitter#getRepairAssignments(boolean, List)} which + * can iterate over this plan in order to generate {@link RepairAssignment}s. + */ +public class PrioritizedRepairPlan +{ + private final int priority; + + private final List keyspaceRepairPlans; + + public PrioritizedRepairPlan(int priority, List keyspaceRepairPlans) + { + this.priority = priority; + this.keyspaceRepairPlans = keyspaceRepairPlans; + } + + public int getPriority() + { + return priority; + } + + public List getKeyspaceRepairPlans() + { + return keyspaceRepairPlans; + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + PrioritizedRepairPlan that = (PrioritizedRepairPlan) o; + return priority == that.priority && Objects.equals(keyspaceRepairPlans, that.keyspaceRepairPlans); + } + + @Override + public int hashCode() + { + return Objects.hash(priority, keyspaceRepairPlans); + } + + @Override + public String toString() + { + return "PrioritizedRepairPlan{" + + "priority=" + priority + + ", keyspaceRepairPlans=" + keyspaceRepairPlans + + '}'; + } + + /** + * Builds a list of {@link PrioritizedRepairPlan}s for the given keyspace and table map, ordered by priority from + * highest to lowest, where priority is derived from table schema's defined priority for the given repair type. + *

+ * If a keyspace has tables with differing priorities, those tables will be included in the PrioritizedRepairPlan + * for their given priority. + * + * @param keyspacesToTableNames A mapping keyspace to table names + * @param repairType The repair type that is being executed + * @param orderFunc A function to order keyspace and tables in the returned plan. + * @return Ordered list of plan's by table priorities. + */ + public static List build(Map> keyspacesToTableNames, AutoRepairConfig.RepairType repairType, Consumer> orderFunc, boolean primaryRangeOnly) + { + // Build a map of priority -> (keyspace -> tables) + Map>> plans = new HashMap<>(); + for (Map.Entry> keyspaceToTableNames : keyspacesToTableNames.entrySet()) + { + String keyspaceName = keyspaceToTableNames.getKey(); + for (String tableName : keyspaceToTableNames.getValue()) + { + int priority = getPriority(repairType, keyspaceName, tableName); + Map> keyspacesForPriority = plans.computeIfAbsent(priority, p -> new HashMap<>()); + List tableNamesAtPriority = keyspacesForPriority.computeIfAbsent(keyspaceName, k -> new ArrayList<>()); + tableNamesAtPriority.add(tableName); + } + } + + // Extract map into a List ordered by priority from highest to lowest. + List planList = new ArrayList<>(plans.size()); + TreeSet priorities = new TreeSet<>(Comparator.reverseOrder()); + priorities.addAll(plans.keySet()); + for (int priority : priorities) + { + Map> keyspacesAndTables = plans.get(priority); + List keyspaceRepairPlans = new ArrayList<>(keyspacesAndTables.size()); + planList.add(new PrioritizedRepairPlan(priority, keyspaceRepairPlans)); + + // Order keyspace and table names based on the input function (typically, this would shuffle the keyspace + // and table names randomly). + List keyspaceNames = new ArrayList<>(keyspacesAndTables.keySet()); + orderFunc.accept(keyspaceNames); + + for (String keyspaceName : keyspaceNames) + { + List tableNames = keyspacesAndTables.get(keyspaceName); + orderFunc.accept(tableNames); + KeyspaceRepairPlan keyspaceRepairPlan = + new KeyspaceRepairPlan(keyspaceName, new ArrayList<>(tableNames), + AutoRepairUtils.calcTotalBytesToBeRepaired(repairType, keyspaceName, tableNames, AutoRepairUtils.getTokenRanges(primaryRangeOnly, keyspaceName))); + keyspaceRepairPlans.add(keyspaceRepairPlan); + } + } + + return planList; + } + + /** + * @return The priority of the given table if defined, otherwise 0. + */ + private static int getPriority(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName) + { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + return cfs != null ? cfs.metadata().params.autoRepair.priority() : 0; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java b/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java new file mode 100644 index 000000000000..6e07399aad7d --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.List; +import java.util.Objects; + +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; + +/** + * Defines a repair assignment to be issued by the autorepair framework. + */ +public class RepairAssignment +{ + final Range tokenRange; + + final String keyspaceName; + + final List tableNames; + + protected final long estimatedBytes; + + public RepairAssignment(Range tokenRange, String keyspaceName, List tableNames, long estimatedBytes) + { + this.tokenRange = tokenRange; + this.keyspaceName = keyspaceName; + this.tableNames = tableNames; + this.estimatedBytes = estimatedBytes; + } + + public Range getTokenRange() + { + return tokenRange; + } + + public String getKeyspaceName() + { + return keyspaceName; + } + + public List getTableNames() + { + return tableNames; + } + + public long getEstimatedBytes() + { + return estimatedBytes; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RepairAssignment that = (RepairAssignment) o; + return Objects.equals(tokenRange, that.tokenRange) && Objects.equals(keyspaceName, that.keyspaceName) + && Objects.equals(tableNames, that.tableNames) && Objects.equals(estimatedBytes, that.estimatedBytes); + } + + @Override + public int hashCode() + { + return Objects.hash(tokenRange, keyspaceName, tableNames, estimatedBytes); + } + + @Override + public String toString() + { + return "RepairAssignment{" + + "tokenRange=" + tokenRange + + ", keyspaceName='" + keyspaceName + '\'' + + ", tableNames=" + tableNames + + ", estimatedBytes=" + estimatedBytes + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairAssignmentIterator.java b/src/java/org/apache/cassandra/repair/autorepair/RepairAssignmentIterator.java new file mode 100644 index 000000000000..44d9f5ef5e55 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairAssignmentIterator.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * Convenience {@link Iterator} implementation to assist implementations of + * {@link IAutoRepairTokenRangeSplitter#getRepairAssignments(boolean, List)} by passing {@link KeyspaceRepairPlan} + * to a custom {@link #next(int, KeyspaceRepairPlan)} method in priority order. + */ +public abstract class RepairAssignmentIterator implements Iterator +{ + private final Iterator repairPlanIterator; + + private Iterator currentIterator = null; + private PrioritizedRepairPlan currentPlan = null; + + public RepairAssignmentIterator(List repairPlans) + { + this.repairPlanIterator = repairPlans.iterator(); + } + + private synchronized Iterator currentIterator() + { + if (currentIterator == null || !currentIterator.hasNext()) + { + // Advance the repair plan iterator if the current repair plan is exhausted, but only + // if there are more repair plans. + if (repairPlanIterator.hasNext()) + { + currentPlan = repairPlanIterator.next(); + currentIterator = currentPlan.getKeyspaceRepairPlans().iterator(); + } + } + return currentIterator; + } + + @Override + public boolean hasNext() + { + Iterator iterator = currentIterator(); + return (iterator != null && iterator.hasNext()); + } + + @Override + public KeyspaceRepairAssignments next() + { + if (!hasNext()) + { + throw new NoSuchElementException("No remaining repair plans"); + } + + final KeyspaceRepairPlan repairPlan = currentIterator().next(); + return next(currentPlan.getPriority(), repairPlan); + } + + /** + * Invoked by {@link #next()} with the next {@link KeyspaceRepairPlan} for the given priority. + * @param priority current priority being processed. + * @param repairPlan the next keyspace repair plan to process + * @return assignments for the given keyspace at this priority. Should never return null, if one desires to + * short-circuit the iterator, override {@link #hasNext()}. + */ + protected abstract KeyspaceRepairAssignments next(int priority, KeyspaceRepairPlan repairPlan); +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairTokenRangeSplitter.java new file mode 100644 index 000000000000..4e44b06208b3 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairTokenRangeSplitter.java @@ -0,0 +1,810 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumMap; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DataStorageSpec; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.lifecycle.SSTableIntervalTree; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.db.lifecycle.View; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.utils.concurrent.Refs; + +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.split; + +/** + * The default implementation of {@link IAutoRepairTokenRangeSplitter} that attempts to: + *

    + *
  1. Create smaller, consistent repair times
  2. + *
  3. Minimize the impact on hosts
  4. + *
  5. Reduce overstreaming
  6. + *
  7. Reduce number of repairs
  8. + *
+ *

+ * To achieve these goals, this implementation inspects SSTable metadata to estimate the bytes and number of partitions + * within a range and splits it accordingly to bound the size of the token ranges used for repair assignments. + *

+ *

+ * Refer to + * Auto Repair documentation for this implementation + * for a more thorough breakdown of this implementation. + *

+ *

+ * While this splitter has a lot of tuning parameters, the expectation is that the established default configuration + * shall be sensible for all {@link org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType}'s. The following + * configuration parameters are offered. + *

+ * + *

Configuration parameters:

+ *
    + *
  • bytes_per_assignment – Target size (in compressed bytes) for each repair. Throttles incremental repair + * and anticompaction per schedule after incremental repairs are enabled.
  • + * + *
  • max_bytes_per_schedule – Maximum data (in compressed bytes) to cover in a single schedule. Acts as a + * throttle for the repair cycle workload. Tune this up if writes are outpacing repair, or down if repairs are too + * disruptive. Alternatively, adjust {@code min_repair_interval}.
  • + * + *
  • partitions_per_assignment – Maximum number of partitions per repair assignment. Limits the number of + * partitions in Merkle tree leaves to prevent overstreaming.
  • + * + *
  • max_tables_per_assignment – Maximum number of tables to include in a single repair assignment. + * Especially useful for keyspaces with many tables. Prevents excessive batching of tables that exceed other + * parameters like {@code bytes_per_assignment} or {@code partitions_per_assignment}.
  • + *
+ */ +public class RepairTokenRangeSplitter implements IAutoRepairTokenRangeSplitter +{ + private static final Logger logger = LoggerFactory.getLogger(RepairTokenRangeSplitter.class); + + // Default max bytes to 100TiB, which is much more readable than Long.MAX_VALUE + private static final DataStorageSpec.LongBytesBound MAX_BYTES = new DataStorageSpec.LongBytesBound(102_400, DataStorageSpec.DataStorageUnit.GIBIBYTES); + + /** + * The target bytes that should be included in a repair assignment + */ + static final String BYTES_PER_ASSIGNMENT = "bytes_per_assignment"; + + /** + * Maximum number of partitions to include in a repair assignment + */ + static final String PARTITIONS_PER_ASSIGNMENT = "partitions_per_assignment"; + + /** + * Maximum number of tables to include in a repair assignment if {@link AutoRepairConfig.Options#repair_by_keyspace} + * is enabled + */ + static final String MAX_TABLES_PER_ASSIGNMENT = "max_tables_per_assignment"; + + /** + * The maximum number of bytes to cover in an individual schedule + */ + static final String MAX_BYTES_PER_SCHEDULE = "max_bytes_per_schedule"; + + static final List PARAMETERS = Arrays.asList(BYTES_PER_ASSIGNMENT, PARTITIONS_PER_ASSIGNMENT, MAX_TABLES_PER_ASSIGNMENT, MAX_BYTES_PER_SCHEDULE); + + private final AutoRepairConfig.RepairType repairType; + + private final Map givenParameters = new HashMap<>(); + + private DataStorageSpec.LongBytesBound bytesPerAssignment; + private long partitionsPerAssignment; + private int maxTablesPerAssignment; + private DataStorageSpec.LongBytesBound maxBytesPerSchedule; + + /** + * Established default for each {@link org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType}, meant to + * choose sensible defaults for each. + *

+ * Defaults if not specified for the given repair type: + *

  • + *
      bytes_per_assignment: 50GiB
    + *
      partitions_per_assignment: 1048576 (2^20)
    + *
      max_tables_per_assignment: 64
    + *
      max_bytes_per_schedule: 1000GiB
    + *
  • + * It's expected that these defaults should work well for everything except incremental, where we set + * max_bytes_per_schedule to 100GiB. This should strike a good balance between the amount of data that will be + * repaired during an initial migration to incremental repair and should move the entire repaired set from + * unrepaired to repaired at steady state, assuming not more the 100GiB of data is written to a node per + * min_repair_interval. + */ + private static final Map DEFAULTS_BY_REPAIR_TYPE = new EnumMap(AutoRepairConfig.RepairType.class) {{ + put(AutoRepairConfig.RepairType.FULL, RepairTypeDefaults.builder(AutoRepairConfig.RepairType.FULL) + .build()); + // Restrict incremental repair to 100GiB max bytes per schedule to confine the amount of possible autocompaction. + put(AutoRepairConfig.RepairType.INCREMENTAL, RepairTypeDefaults.builder(AutoRepairConfig.RepairType.INCREMENTAL) + .withMaxBytesPerSchedule(new DataStorageSpec.LongBytesBound("100GiB")) + .build()); + put(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, RepairTypeDefaults.builder(AutoRepairConfig.RepairType.PREVIEW_REPAIRED) + .build()); + }}; + + public RepairTokenRangeSplitter(AutoRepairConfig.RepairType repairType, Map parameters) + { + this.repairType = repairType; + this.givenParameters.putAll(parameters); + + reinitParameters(); + } + + private void reinitParameters() + { + RepairTypeDefaults defaults = DEFAULTS_BY_REPAIR_TYPE.get(repairType); + + DataStorageSpec.LongBytesBound bytesPerAssignmentTmp = getPropertyOrDefault(BYTES_PER_ASSIGNMENT, DataStorageSpec.LongBytesBound::new, defaults.bytesPerAssignment); + DataStorageSpec.LongBytesBound maxBytesPerScheduleTmp = getPropertyOrDefault(MAX_BYTES_PER_SCHEDULE, DataStorageSpec.LongBytesBound::new, defaults.maxBytesPerSchedule); + + // Validate that bytesPerAssignment <= maxBytesPerSchedule + if (bytesPerAssignmentTmp.toBytes() > maxBytesPerScheduleTmp.toBytes()) + { + throw new IllegalArgumentException(String.format("%s='%s' cannot be greater than %s='%s' for %s", + BYTES_PER_ASSIGNMENT, + bytesPerAssignmentTmp, + MAX_BYTES_PER_SCHEDULE, + maxBytesPerScheduleTmp, + repairType.getConfigName())); + } + + bytesPerAssignment = bytesPerAssignmentTmp; + maxBytesPerSchedule = maxBytesPerScheduleTmp; + + partitionsPerAssignment = getPropertyOrDefault(PARTITIONS_PER_ASSIGNMENT, Long::parseLong, defaults.partitionsPerAssignment); + maxTablesPerAssignment = getPropertyOrDefault(MAX_TABLES_PER_ASSIGNMENT, Integer::parseInt, defaults.maxTablesPerAssignment); + + logger.info("Configured {}[{}] with {}={}, {}={}, {}={}, {}={}", RepairTokenRangeSplitter.class.getName(), + repairType.getConfigName(), + BYTES_PER_ASSIGNMENT, bytesPerAssignment, + PARTITIONS_PER_ASSIGNMENT, partitionsPerAssignment, + MAX_TABLES_PER_ASSIGNMENT, maxTablesPerAssignment, + MAX_BYTES_PER_SCHEDULE, maxBytesPerSchedule); + } + + private T getPropertyOrDefault(String propertyName, Function mapper, T defaultValue) + { + return Optional.ofNullable(this.givenParameters.get(propertyName)).map(mapper).orElse(defaultValue); + } + + @Override + public Iterator getRepairAssignments(boolean primaryRangeOnly, List repairPlans) + { + return new BytesBasedRepairAssignmentIterator(primaryRangeOnly, repairPlans); + } + + /** + * A custom {@link RepairAssignmentIterator} that confines the number of repair assignments to + * max_bytes_per_schedule. + */ + private class BytesBasedRepairAssignmentIterator extends RepairAssignmentIterator + { + + private final boolean primaryRangeOnly; + private long bytesSoFar = 0; + + BytesBasedRepairAssignmentIterator(boolean primaryRangeOnly, List repairPlans) + { + super(repairPlans); + this.primaryRangeOnly = primaryRangeOnly; + } + + @Override + protected KeyspaceRepairAssignments next(int priority, KeyspaceRepairPlan repairPlan) + { + // short circuit if we've accumulated too many bytes by returning a KeyspaceRepairAssignments with + // no assignments. We do this rather than returning false in hasNext() because we want to signal + // to AutoRepair that a keyspace generated no assignments. + if (bytesSoFar >= maxBytesPerSchedule.toBytes()) + { + return new KeyspaceRepairAssignments(priority, repairPlan.getKeyspaceName(), Collections.emptyList()); + } + + List> tokenRanges = AutoRepairUtils.getTokenRanges(primaryRangeOnly, repairPlan.getKeyspaceName()); + // shuffle token ranges to unbias selection of ranges + Collections.shuffle(tokenRanges); + List repairAssignments = new ArrayList<>(); + // Generate assignments for each range speparately + for (Range tokenRange : tokenRanges) + { + repairAssignments.addAll(getRepairAssignmentsForKeyspace(repairType, repairPlan, tokenRange)); + } + + FilteredRepairAssignments filteredRepairAssignments = filterRepairAssignments(priority, repairPlan.getKeyspaceName(), repairAssignments, bytesSoFar); + bytesSoFar = filteredRepairAssignments.newBytesSoFar; + return new KeyspaceRepairAssignments(priority, repairPlan.getKeyspaceName(), filteredRepairAssignments.repairAssignments); + } + } + + @VisibleForTesting + List getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType repairType, KeyspaceRepairPlan repairPlan, Range tokenRange) + { + List repairAssignments = new ArrayList<>(); + // this is used for batching minimal single assignment tables together + List currentAssignments = new ArrayList<>(); + + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + + // If we can repair by keyspace, sort the tables by size so can batch the smallest ones together + boolean repairByKeyspace = config.getRepairByKeyspace(repairType); + List tablesToProcess = repairPlan.getTableNames(); + if (repairByKeyspace) + { + tablesToProcess = repairPlan.getTableNames().stream().sorted((t1, t2) -> { + ColumnFamilyStore cfs1 = ColumnFamilyStore.getIfExists(repairPlan.getKeyspaceName(), t1); + ColumnFamilyStore cfs2 = ColumnFamilyStore.getIfExists(repairPlan.getKeyspaceName(), t2); + // If for whatever reason the CFS is not retrievable, we can assume it has been deleted, so give the + // other cfs precedence. + if (cfs1 == null) + { + // cfs1 is lesser than because its null + return -1; + } + else if (cfs2 == null) + { + // cfs1 is greather than because cfs2 is null + return 1; + } + return Long.compare(cfs1.metric.totalDiskSpaceUsed.getCount(), cfs2.metric.totalDiskSpaceUsed.getCount()); + }).collect(Collectors.toList()); + } + + for (String tableName : tablesToProcess) + { + List tableAssignments = getRepairAssignmentsForTable(repairPlan, tableName, tokenRange); + + if (tableAssignments.isEmpty()) + continue; + + // if not repairing by keyspace don't attempt to batch them with others. + if (!repairByKeyspace) + { + repairAssignments.addAll(tableAssignments); + } + // If the table assignments are for the same token range, and we have room to add more tables to the current assignment + else if (tableAssignments.size() == 1 && + currentAssignments.size() < maxTablesPerAssignment && + (currentAssignments.isEmpty() || currentAssignments.get(0).getTokenRange().equals(tableAssignments.get(0).getTokenRange()))) + { + long currentAssignmentsBytes = getEstimatedBytes(currentAssignments); + long tableAssignmentsBytes = getEstimatedBytes(tableAssignments); + // only add assignments together if they don't exceed max bytes per schedule. + if (currentAssignmentsBytes + tableAssignmentsBytes < maxBytesPerSchedule.toBytes()) { + currentAssignments.addAll(tableAssignments); + } + else + { + // add table assignments by themselves + repairAssignments.addAll(tableAssignments); + } + } + else + { + if (!currentAssignments.isEmpty()) + { + repairAssignments.add(merge(currentAssignments)); + currentAssignments.clear(); + } + repairAssignments.addAll(tableAssignments); + } + } + + if (!currentAssignments.isEmpty()) + repairAssignments.add(merge(currentAssignments)); + + return repairAssignments; + } + + /** + * Given a repair type and map of sized-based repair assignments, confine them by maxBytesPerSchedule. + * + * @param repairAssignments the assignments to filter. + * @param bytesSoFar repair assignment bytes accumulated so far. + * @return A list of repair assignments confined by maxBytesPerSchedule. + */ + @VisibleForTesting + FilteredRepairAssignments filterRepairAssignments(int priority, String keyspaceName, List repairAssignments, long bytesSoFar) + { + // Confine repair assignments by maxBytesPerSchedule. + long bytesSoFarThisIteration = 0L; + long bytesNotRepaired = 0L; + int assignmentsNotRepaired = 0; + int assignmentsToRepair = 0; + int totalAssignments = 0; + + List assignmentsToReturn = new ArrayList<>(repairAssignments.size()); + for (SizedRepairAssignment repairAssignment : repairAssignments) + { + totalAssignments++; + // skip any repair assignments that would accumulate us past the maxBytesPerSchedule + if (bytesSoFar + repairAssignment.getEstimatedBytes() > maxBytesPerSchedule.toBytes()) + { + // log that repair assignment was skipped. + bytesNotRepaired += repairAssignment.getEstimatedBytes(); + assignmentsNotRepaired++; + logger.warn("Skipping {} because it would increase total repair bytes to {}", + repairAssignment, + getBytesOfMaxBytesPerSchedule(bytesSoFar + repairAssignment.getEstimatedBytes())); + } + else + { + bytesSoFar += repairAssignment.getEstimatedBytes(); + bytesSoFarThisIteration += repairAssignment.getEstimatedBytes(); + assignmentsToRepair++; + logger.info("Adding {}, increasing repair bytes to {}", + repairAssignment, + getBytesOfMaxBytesPerSchedule(bytesSoFar)); + assignmentsToReturn.add(repairAssignment); + } + } + + String message = "Returning {} assignment(s) for priorityBucket {} and keyspace {}, totaling {} ({} overall)"; + if (assignmentsNotRepaired != 0) + { + message += ". Skipping {} of {} assignment(s), totaling {}"; + if (repairType != AutoRepairConfig.RepairType.INCREMENTAL) + { + message += ". The entire primary range will not be repaired this schedule. " + + "Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure " + + "all ranges do get repaired within gc_grace_seconds"; + logger.warn(message, assignmentsToRepair, priority, keyspaceName, + FileUtils.stringifyFileSize(bytesSoFarThisIteration), + getBytesOfMaxBytesPerSchedule(bytesSoFar), + assignmentsNotRepaired, totalAssignments, + FileUtils.stringifyFileSize(bytesNotRepaired)); + } + else + { + logger.info(message, assignmentsToRepair, priority, keyspaceName, + FileUtils.stringifyFileSize(bytesSoFarThisIteration), + getBytesOfMaxBytesPerSchedule(bytesSoFar), + assignmentsNotRepaired, totalAssignments, + FileUtils.stringifyFileSize(bytesNotRepaired)); + } + } + else + { + logger.info(message, assignmentsToRepair, priority, keyspaceName, + FileUtils.stringifyFileSize(bytesSoFarThisIteration), + getBytesOfMaxBytesPerSchedule(bytesSoFar)); + } + + return new FilteredRepairAssignments(assignmentsToReturn, bytesSoFar); + } + + @VisibleForTesting + static class FilteredRepairAssignments + { + final List repairAssignments; + final long newBytesSoFar; + + private FilteredRepairAssignments(List repairAssignments, long newBytesSoFar) + { + this.repairAssignments = repairAssignments; + this.newBytesSoFar = newBytesSoFar; + } + } + + private String getBytesOfMaxBytesPerSchedule(long bytes) + { + if (maxBytesPerSchedule.equals(MAX_BYTES)) + return FileUtils.stringifyFileSize(bytes); + else + return String.format("%s of %s", FileUtils.stringifyFileSize(bytes), maxBytesPerSchedule); + } + + /** + * @param repairAssignments The assignments to sum + * @return The sum of {@link SizedRepairAssignment#getEstimatedBytes()} of all given + * repairAssignments. + */ + @VisibleForTesting + protected static long getEstimatedBytes(List repairAssignments) + { + return repairAssignments + .stream() + .mapToLong(SizedRepairAssignment::getEstimatedBytes) + .sum(); + } + + @VisibleForTesting + static SizedRepairAssignment merge(List assignments) + { + if (assignments.isEmpty()) + throw new IllegalStateException("Cannot merge empty assignments"); + + Set mergedTableNames = new HashSet<>(); + Range referenceTokenRange = assignments.get(0).getTokenRange(); + String referenceKeyspaceName = assignments.get(0).getKeyspaceName(); + + for (SizedRepairAssignment assignment : assignments) + { + // These checks _should_ be unnecessary but are here to ensure that the assignments are consistent + if (!assignment.getTokenRange().equals(referenceTokenRange)) + throw new IllegalStateException("All assignments must have the same token range"); + if (!assignment.getKeyspaceName().equals(referenceKeyspaceName)) + throw new IllegalStateException("All assignments must have the same keyspace name"); + + mergedTableNames.addAll(assignment.getTableNames()); + } + + long sizeForAssignment = getEstimatedBytes(assignments); + return new SizedRepairAssignment(referenceTokenRange, referenceKeyspaceName, new ArrayList<>(mergedTableNames), + "full primary range for " + mergedTableNames.size() + " tables", sizeForAssignment); + } + + @VisibleForTesting + protected List getRepairAssignmentsForTable(KeyspaceRepairPlan repairPlan, String tableName, Range tokenRange) + { + AutoRepairUtils.SizeEstimate sizeEstimate = repairPlan.getSizeEstimate(AutoRepairUtils.getKeyspaceTableName(repairPlan.getKeyspaceName(), tableName), tokenRange); + if (sizeEstimate == null) + { + // Ideally, it should have been cached already inside the KeyspaceRepairPlan, but incase it was not, + // then recalculating it. It is a bit expensive, but necessary for the repair + logger.warn("The size estimate for {}.{} range {} was not pre-calculated, calculating on-demand", + repairPlan.getKeyspaceName(), tableName, tokenRange); + sizeEstimate = AutoRepairUtils.getRangeSizeEstimate(repairType, repairPlan.getKeyspaceName(), tableName, tokenRange); + } + return getRepairAssignments(sizeEstimate); + } + + private static void logSkippingTable(String keyspaceName, String tableName) + { + logger.warn("Could not resolve table data for {}.{} assuming it has since been deleted, skipping", keyspaceName, tableName); + } + + @VisibleForTesting + protected List getRepairAssignments(AutoRepairUtils.SizeEstimate estimate) + { + List repairAssignments = new ArrayList<>(); + + // since its possible for us to hit maxBytesPerSchedule before seeing all ranges, shuffle so there is chance + // at least of hitting all the ranges _eventually_ for the worst case scenarios + int totalExpectedSubRanges = 0; + if (estimate.sizeForRepair != 0) + { + boolean needsSplitting = estimate.sizeForRepair > bytesPerAssignment.toBytes() || estimate.partitions > partitionsPerAssignment; + if (needsSplitting) + { + totalExpectedSubRanges += calculateNumberOfSplits(estimate); + } + } + if (estimate.sizeForRepair == 0) + { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(estimate.keyspace, estimate.table); + + if (cfs == null) + { + logSkippingTable(estimate.keyspace, estimate.table); + return Collections.emptyList(); + } + + long memtableSize = cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize(); + if (memtableSize > 0L) + { + logger.debug("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", estimate.keyspace, estimate.table, estimate.tokenRange, memtableSize); + SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), "full primary rangee for table with memtable only detected", memtableSize); + repairAssignments.add(assignment); + } + else + { + logger.debug("Included {}.{} range {}, has no SSTables or memtable data, but adding single repair assignment for entire range in case writes were missed", estimate.keyspace, estimate.table, estimate.tokenRange); + SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), "full primary range for table with no data detected", 0L); + repairAssignments.add(assignment); + } + } + else + { + // Check if the estimate needs splitting based on the criteria + boolean needsSplitting = estimate.sizeForRepair > bytesPerAssignment.toBytes() || estimate.partitions > partitionsPerAssignment; + if (needsSplitting) + { + int numberOfSplits = calculateNumberOfSplits(estimate); + long approximateBytesPerSplit = estimate.sizeForRepair / numberOfSplits; + Collection> subranges = split(estimate.tokenRange, numberOfSplits); + for (Range subrange : subranges) + { + SizedRepairAssignment assignment = new SizedRepairAssignment(subrange, estimate.keyspace, Collections.singletonList(estimate.table), + String.format("subrange %d of %d", repairAssignments.size() + 1, totalExpectedSubRanges), + approximateBytesPerSplit); + repairAssignments.add(assignment); + } + } + else + { + // No splitting needed, repair the entire range as-is + SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, + Collections.singletonList(estimate.table), + "full primary range for table", estimate.sizeForRepair); + repairAssignments.add(assignment); + } + } + return repairAssignments; + } + + private int calculateNumberOfSplits(AutoRepairUtils.SizeEstimate estimate) + { + // Calculate the number of splits needed for size and partitions + int splitsForSize = (int) Math.ceil((double) estimate.sizeForRepair / bytesPerAssignment.toBytes()); + int splitsForPartitions = (int) Math.ceil((double) estimate.partitions / partitionsPerAssignment); + + // Split the token range into subranges based on whichever (partitions, bytes) would generate the most splits. + boolean splitBySize = splitsForSize > splitsForPartitions; + int splits = splitBySize ? splitsForSize : splitsForPartitions; + + // calculate approximation for logging purposes + long approximateBytesPerSplit = estimate.sizeForRepair / splits; + long approximatePartitionsPerSplit = estimate.partitions / splits; + + logger.info("Splitting {}.{} for range {} into {} sub ranges by {} (splitsForSize={}, splitsForPartitions={}, " + + "approximateBytesInRange={}, approximatePartitionsInRange={}, " + + "approximateBytesPerSplit={}, approximatePartitionsPerSplit={})", + estimate.keyspace, estimate.table, estimate.tokenRange, + splits, splitBySize ? "size" : "partitions", + splitsForSize, splitsForPartitions, + FileUtils.stringifyFileSize(estimate.sizeForRepair), estimate.partitions, + FileUtils.stringifyFileSize(approximateBytesPerSplit), approximatePartitionsPerSplit + ); + return splits; + } + + @VisibleForTesting + static Refs getSSTableReaderRefs(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName, Range tokenRange) + { + final ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + if (cfs == null) + { + logSkippingTable(keyspaceName, tableName); + return Refs.ref(Collections.emptyList()); + } + + Refs refs = null; + while (refs == null) + { + Iterable sstables = cfs.getTracker().getView().select(SSTableSet.CANONICAL); + SSTableIntervalTree tree = SSTableIntervalTree.buildSSTableIntervalTree(Lists.newArrayList(sstables)); + Range r = Range.makeRowRange(tokenRange); + List canonicalSSTables = View.sstablesInBounds(r.left, r.right, tree); + if (repairType == AutoRepairConfig.RepairType.INCREMENTAL) + { + canonicalSSTables = canonicalSSTables.stream().filter((sstable) -> !sstable.isRepaired()).collect(Collectors.toList()); + } + refs = Refs.tryRef(canonicalSSTables); + } + return refs; + } + + @Override + public void setParameter(String key, String value) + { + if (!PARAMETERS.contains(key)) + { + throw new IllegalArgumentException("Unexpected parameter '" + key + "', must be one of " + PARAMETERS); + } + + logger.info("Setting {} to {} for repair type {}", key, value, repairType); + givenParameters.put(key, value); + reinitParameters(); + } + + @Override + public Map getParameters() + { + final Map parameters = new LinkedHashMap<>(); + for (String parameter : PARAMETERS) + { + // Use the parameter as provided if present. + if (givenParameters.containsKey(parameter)) + { + parameters.put(parameter, givenParameters.get(parameter)); + continue; + } + + switch (parameter) + { + case BYTES_PER_ASSIGNMENT: + parameters.put(parameter, bytesPerAssignment.toString()); + continue; + case PARTITIONS_PER_ASSIGNMENT: + parameters.put(parameter, Long.toString(partitionsPerAssignment)); + continue; + case MAX_TABLES_PER_ASSIGNMENT: + parameters.put(parameter, Integer.toString(maxTablesPerAssignment)); + continue; + case MAX_BYTES_PER_SCHEDULE: + parameters.put(parameter, maxBytesPerSchedule.toString()); + continue; + default: + // not expected + parameters.put(parameter, ""); + } + } + return Collections.unmodifiableMap(parameters); + } + + /** + * Implementation of RepairAssignment that also assigns an estimation of bytes involved + * in the repair. + */ + @VisibleForTesting + protected static class SizedRepairAssignment extends RepairAssignment + { + + final String description; + + public SizedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames) + { + this(tokenRange, keyspaceName, tableNames, "", 0L); + } + + public SizedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames, + String description, + long estimatedBytes) + { + super(tokenRange, keyspaceName, tableNames, estimatedBytes); + this.description = description; + } + + /** + * @return Additional metadata about the repair assignment. + */ + public String getDescription() { + return description; + } + + /** + * Estimated bytes involved in the assignment. Typically Derived from {@link AutoRepairUtils.SizeEstimate#sizeForRepair}. + * + * @return estimated bytes involved in the assignment. + */ + public long getEstimatedBytes() + { + return estimatedBytes; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + SizedRepairAssignment that = (SizedRepairAssignment) o; + return estimatedBytes == that.estimatedBytes && Objects.equals(description, that.description); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), description, estimatedBytes); + } + + @Override + public String toString() + { + return "SizedRepairAssignment{" + + "description='" + description + '\'' + + ", tokenRange=" + tokenRange + + ", keyspaceName='" + keyspaceName + '\'' + + ", tableNames=" + tableNames + + ", estimatedBytes=" + FileUtils.stringifyFileSize(estimatedBytes) + + '}'; + } + } + + /** + * Conveinence builder for establishing defaults by repair type. + */ + protected static class RepairTypeDefaults + { + final AutoRepairConfig.RepairType repairType; + final DataStorageSpec.LongBytesBound bytesPerAssignment; + final long partitionsPerAssignment; + final int maxTablesPerAssignment; + final DataStorageSpec.LongBytesBound maxBytesPerSchedule; + + public RepairTypeDefaults(AutoRepairConfig.RepairType repairType, + DataStorageSpec.LongBytesBound bytesPerAssignment, + long partitionsPerAssignment, + int maxTablesPerAssignment, + DataStorageSpec.LongBytesBound maxBytesPerSchedule) + { + this.repairType = repairType; + this.bytesPerAssignment = bytesPerAssignment; + this.partitionsPerAssignment = partitionsPerAssignment; + this.maxTablesPerAssignment = maxTablesPerAssignment; + this.maxBytesPerSchedule = maxBytesPerSchedule; + } + + static RepairTypeDefaultsBuilder builder(AutoRepairConfig.RepairType repairType) + { + return new RepairTypeDefaultsBuilder(repairType); + } + + static class RepairTypeDefaultsBuilder + { + private final AutoRepairConfig.RepairType repairType; + private DataStorageSpec.LongBytesBound bytesPerAssignment = new DataStorageSpec.LongBytesBound("50GiB"); + // Aims to target at most 1 partitions per leaf assuming a merkle tree of depth 20 (2^20 = 1,048,576) + private long partitionsPerAssignment = 1_048_576; + private int maxTablesPerAssignment = 64; + private DataStorageSpec.LongBytesBound maxBytesPerSchedule = MAX_BYTES; + + private RepairTypeDefaultsBuilder(AutoRepairConfig.RepairType repairType) + { + this.repairType = repairType; + } + + @SuppressWarnings("unused") + public RepairTypeDefaultsBuilder withBytesPerAssignment(DataStorageSpec.LongBytesBound bytesPerAssignment) + { + this.bytesPerAssignment = bytesPerAssignment; + return this; + } + + @SuppressWarnings("unused") + public RepairTypeDefaultsBuilder withPartitionsPerAssignment(long partitionsPerAssignment) + { + this.partitionsPerAssignment = partitionsPerAssignment; + return this; + } + + @SuppressWarnings("unused") + public RepairTypeDefaultsBuilder withMaxTablesPerAssignment(int maxTablesPerAssignment) + { + this.maxTablesPerAssignment = maxTablesPerAssignment; + return this; + } + + public RepairTypeDefaultsBuilder withMaxBytesPerSchedule(DataStorageSpec.LongBytesBound maxBytesPerSchedule) + { + this.maxBytesPerSchedule = maxBytesPerSchedule; + return this; + } + + public RepairTokenRangeSplitter.RepairTypeDefaults build() + { + return new RepairTypeDefaults(repairType, bytesPerAssignment, partitionsPerAssignment, maxTablesPerAssignment, maxBytesPerSchedule); + } + } + } +} diff --git a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java index 855ad4bad344..820c6b011ba6 100644 --- a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java +++ b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java @@ -81,7 +81,7 @@ public String toString() } } - private static class Table + public static class Table { final String keyspace; @@ -94,7 +94,7 @@ private static class Table final Map, Session> sessions = new HashMap<>(); - Table(String keyspace, String table) + public Table(String keyspace, String table) { this.keyspace = keyspace; this.table = table; @@ -138,7 +138,7 @@ void calculateTotals() totalsCalculated = true; } - boolean isCounter() + public boolean isCounter() { TableMetadata tmd = Schema.instance.getTableMetadata(keyspace, table); return tmd != null && tmd.isCounter(); @@ -174,6 +174,16 @@ public String toString() } return output.toString(); } + + public long getBytes() + { + return this.bytes; + } + + public long getRanges() + { + return this.ranges.size(); + } } private final Map, Table> summaries = new HashMap<>(); @@ -233,6 +243,12 @@ private void calculateTotals() totalsCalculated = true; } + public Map, Table> getTotals() + { + calculateTotals(); + return summaries; + } + public String toString() { List> tables = Lists.newArrayList(summaries.keySet()); diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java new file mode 100644 index 000000000000..5f05edab9dd8 --- /dev/null +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.schema; + +import java.util.Arrays; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.StringUtils; + +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; + +import static java.lang.String.format; + +/** + * AutoRepair table parameters - used to define the auto-repair configuration for a table. + */ +public final class AutoRepairParams +{ + public enum Option + { + FULL_ENABLED, + INCREMENTAL_ENABLED, + PREVIEW_REPAIRED_ENABLED, + PRIORITY; + + @Override + public String toString() + { + return name().toLowerCase(); + } + } + + private final ImmutableMap options; + + public static final Map DEFAULT_OPTIONS = ImmutableMap.of( + Option.FULL_ENABLED.name().toLowerCase(), Boolean.toString(true), + Option.INCREMENTAL_ENABLED.name().toLowerCase(), Boolean.toString(true), + Option.PREVIEW_REPAIRED_ENABLED.name().toLowerCase(), Boolean.toString(true), + Option.PRIORITY.toString(), "0" + ); + + AutoRepairParams(Map options) + { + this.options = ImmutableMap.copyOf(options); + } + + public static final AutoRepairParams DEFAULT = + new AutoRepairParams(DEFAULT_OPTIONS); + + public static AutoRepairParams create(Map options) + { + Map optionsMap = new TreeMap<>(DEFAULT_OPTIONS); + if (options != null) + { + for (Map.Entry entry : options.entrySet()) + { + if (Arrays.stream(Option.values()).noneMatch(option -> option.toString().equalsIgnoreCase(entry.getKey()))) + { + throw new ConfigurationException(format("Unknown property '%s'", entry.getKey())); + } + optionsMap.put(entry.getKey(), entry.getValue()); + } + } + return new AutoRepairParams(optionsMap); + } + + public boolean repairEnabled(AutoRepairConfig.RepairType type) + { + String option = type.toString().toLowerCase() + "_enabled"; + String enabled = options.getOrDefault(option, DEFAULT_OPTIONS.get(option)); + return Boolean.parseBoolean(enabled); + } + + public int priority() + { + String priority = options.getOrDefault(Option.PRIORITY.toString(), DEFAULT_OPTIONS.get(Option.PRIORITY.toString())); + return Integer.parseInt(priority); + } + + public void validate() + { + for (Option option : Option.values()) + { + if (!options.containsKey(option.toString().toLowerCase())) + { + throw new ConfigurationException(format("Missing repair sub-option '%s'", option)); + } + } + if (options.get(Option.FULL_ENABLED.toString().toLowerCase()) != null && !isValidBoolean(options.get(Option.FULL_ENABLED.toString().toLowerCase()))) + { + throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", + options.get(Option.FULL_ENABLED.toString().toLowerCase()), + Option.FULL_ENABLED)); + } + if (options.get(Option.INCREMENTAL_ENABLED.toString().toLowerCase()) != null && !isValidBoolean(options.get(Option.INCREMENTAL_ENABLED.toString().toLowerCase()))) + { + throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", + options.get(Option.INCREMENTAL_ENABLED.toString().toLowerCase()), + Option.INCREMENTAL_ENABLED)); + } + if (options.get(Option.PREVIEW_REPAIRED_ENABLED.toString().toLowerCase()) != null && !isValidBoolean(options.get(Option.PREVIEW_REPAIRED_ENABLED.toString().toLowerCase()))) + { + throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", + options.get(Option.PREVIEW_REPAIRED_ENABLED.toString().toLowerCase()), + Option.PREVIEW_REPAIRED_ENABLED)); + } + if (options.get(Option.PRIORITY.toString().toLowerCase()) != null && !isValidInt(options.get(Option.PRIORITY.toString().toLowerCase()))) + { + throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be an integer", + options.get(Option.PRIORITY.toString().toLowerCase()), + Option.PRIORITY)); + } + } + + public static boolean isValidBoolean(String value) + { + return StringUtils.equalsIgnoreCase(value, "true") || StringUtils.equalsIgnoreCase(value, "false"); + } + + public static boolean isValidInt(String value) + { + return StringUtils.isNumeric(value); + } + + public Map options() + { + return options; + } + + public static AutoRepairParams fromMap(Map map) + { + return create(map); + } + + public Map asMap() + { + return options; + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("options", options) + .toString(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof AutoRepairParams)) + return false; + + AutoRepairParams cp = (AutoRepairParams) o; + + return options.equals(cp.options); + } + + @Override + public int hashCode() + { + return Objects.hash(options); + } +} diff --git a/src/java/org/apache/cassandra/schema/SchemaConstants.java b/src/java/org/apache/cassandra/schema/SchemaConstants.java index ff990f7cbb06..614ba9909206 100644 --- a/src/java/org/apache/cassandra/schema/SchemaConstants.java +++ b/src/java/org/apache/cassandra/schema/SchemaConstants.java @@ -186,7 +186,7 @@ public static Set getLocalAndReplicatedSystemTableNames() .addAll(SchemaKeyspaceTables.ALL) .addAll(TraceKeyspace.TABLE_NAMES) .addAll(AuthKeyspace.TABLE_NAMES) - .addAll(SystemDistributedKeyspace.TABLE_NAMES) + .addAll(SystemDistributedKeyspace.getTableNames()) .build(); } } diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index f7044b55f7fa..9aa8e4bdc24d 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -96,6 +96,8 @@ private SchemaKeyspace() + "replication frozen>," + "PRIMARY KEY ((keyspace_name)))"); + // auto_repair column is only included if AUTOREPAIR_ENABLE is true to avoid schema disagreement + // with pre-5.0.7 nodes that don't have this column in their system_schema.tables definition private static final TableMetadata Tables = parse(TABLES, "table definitions", @@ -125,6 +127,7 @@ private SchemaKeyspace() + "additional_write_policy text," + "cdc boolean," + "read_repair text," + + (CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean() ? "auto_repair frozen>," : "") + "PRIMARY KEY ((keyspace_name), table_name))"); private static final TableMetadata Columns = @@ -177,6 +180,8 @@ private SchemaKeyspace() + "options frozen>," + "PRIMARY KEY ((keyspace_name), table_name, trigger_name))"); + // auto_repair column is only included if AUTOREPAIR_ENABLE is true to avoid schema disagreement + // with pre-5.0.7 nodes that don't have this column in their system_schema.views definition private static final TableMetadata Views = parse(VIEWS, "view definitions", @@ -209,6 +214,7 @@ private SchemaKeyspace() + "additional_write_policy text," + "cdc boolean," + "read_repair text," + + (CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean() ? "auto_repair frozen>," : "") + "PRIMARY KEY ((keyspace_name), view_name))"); private static final TableMetadata Indexes = @@ -563,7 +569,12 @@ private static void addTableToSchemaMutation(TableMetadata table, boolean withCo } } - private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBuilder builder) + public static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBuilder builder) + { + addTableParamsToRowBuilder(params, Tables, builder); + } + + private static void addTableParamsToRowBuilder(TableParams params, TableMetadata schemaTable, Row.SimpleBuilder builder) { builder.add("bloom_filter_fp_chance", params.bloomFilterFpChance) .add("comment", params.comment) @@ -602,6 +613,18 @@ private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBui // incremental_backups is enabled, to avoid RTE in pre-4.2 versioned node during upgrades if (!params.incrementalBackups) builder.add("incremental_backups", false); + + // As above, only add the auto_repair column if: + // 1. The column exists in the schema (depends on AUTOREPAIR_ENABLE at class load time) + // 2. The JVM property is enabled AND the scheduler is enabled + // to avoid RTE in pre-5.1 versioned node during upgrades + if (schemaTable.getColumn(ByteBufferUtil.bytes("auto_repair")) != null + && CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean() + && DatabaseDescriptor.getRawConfig() != null + && DatabaseDescriptor.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) + { + builder.add("auto_repair", params.autoRepair.asMap()); + } } private static void addAlterTableToSchemaMutation(TableMetadata oldTable, TableMetadata newTable, Mutation.SimpleBuilder builder) @@ -814,7 +837,7 @@ private static void addViewToSchemaMutation(ViewMetadata view, boolean includeCo .add("where_clause", view.whereClause.toCQLString()) .add("id", table.id.asUUID()); - addTableParamsToRowBuilder(table.params, rowBuilder); + addTableParamsToRowBuilder(table.params, Views, rowBuilder); if (includeColumns) { @@ -1070,6 +1093,12 @@ static TableParams createTableParamsFromRow(UntypedResultSet.Row row) if (row.has("incremental_backups")) builder.incrementalBackups(row.getBoolean("incremental_backups")); + // auto_repair column was introduced in 5.1 + if (row.has("auto_repair")) + { + builder.automatedRepair(AutoRepairParams.fromMap(row.getFrozenTextMap("auto_repair"))); + } + return builder.build(); } diff --git a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java index b36ec64874f3..673351378714 100644 --- a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java @@ -83,8 +83,9 @@ private SystemDistributedKeyspace() * gen 4: compression chunk length reduced to 16KiB, memtable_flush_period_in_ms now unset on all tables in 4.0 * gen 5: add ttl and TWCS to repair_history tables * gen 6: add denylist table + * gen 7: add auto_repair_history and auto_repair_priority tables for AutoRepair feature */ - public static final long GENERATION = 6; + public static final long GENERATION = 7; public static final String REPAIR_HISTORY = "repair_history"; @@ -94,8 +95,27 @@ private SystemDistributedKeyspace() public static final String PARTITION_DENYLIST_TABLE = "partition_denylist"; - public static final Set TABLE_NAMES = ImmutableSet.of(REPAIR_HISTORY, PARENT_REPAIR_HISTORY, VIEW_BUILD_STATUS, PARTITION_DENYLIST_TABLE); - + public static final String AUTO_REPAIR_HISTORY = "auto_repair_history"; + + public static final String AUTO_REPAIR_PRIORITY = "auto_repair_priority"; + + /** + * Returns the set of table names for the system_distributed keyspace. + * When AUTOREPAIR_ENABLE is false, auto-repair tables are excluded to avoid schema changes + * for users who never enable the feature. + */ + public static Set getTableNames() + { + if (CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean()) + { + return ImmutableSet.of(REPAIR_HISTORY, PARENT_REPAIR_HISTORY, VIEW_BUILD_STATUS, + PARTITION_DENYLIST_TABLE, AUTO_REPAIR_HISTORY, AUTO_REPAIR_PRIORITY); + } + return ImmutableSet.of(REPAIR_HISTORY, PARENT_REPAIR_HISTORY, VIEW_BUILD_STATUS, + PARTITION_DENYLIST_TABLE); + } + + private static final TableMetadata RepairHistory = parse(REPAIR_HISTORY, "Repair history", @@ -161,6 +181,27 @@ private SystemDistributedKeyspace() + "PRIMARY KEY ((ks_name, table_name), key))") .build(); + public static final TableMetadata AutoRepairHistory = + parse(AUTO_REPAIR_HISTORY, + "Auto repair history for each node", + "CREATE TABLE %s (" + + "host_id uuid," + + "repair_type text," + + "repair_turn text," + + "repair_start_ts timestamp," + + "repair_finish_ts timestamp," + + "delete_hosts set," + + "delete_hosts_update_time timestamp," + + "force_repair boolean," + + "PRIMARY KEY (repair_type, host_id))").build(); + public static final TableMetadata AutoRepairPriority = + parse(AUTO_REPAIR_PRIORITY, + "Auto repair priority for each group", + "CREATE TABLE %s (" + + "repair_type text," + + "repair_priority set," + + "PRIMARY KEY (repair_type))").build(); + private static TableMetadata.Builder parse(String table, String description, String cql) { return CreateTableStatement.parse(format(cql, table), SchemaConstants.DISTRIBUTED_KEYSPACE_NAME) @@ -170,7 +211,20 @@ private static TableMetadata.Builder parse(String table, String description, Str public static KeyspaceMetadata metadata() { - return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, KeyspaceParams.simple(Math.max(DEFAULT_RF, DatabaseDescriptor.getDefaultKeyspaceRF())), Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, PartitionDenylistTable)); + Tables tables; + if (CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean()) + { + tables = Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, + PartitionDenylistTable, AutoRepairHistory, AutoRepairPriority); + } + else + { + tables = Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, + PartitionDenylistTable); + } + return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + KeyspaceParams.simple(Math.max(DEFAULT_RF, DatabaseDescriptor.getDefaultKeyspaceRF())), + tables); } public static void startParentRepair(TimeUUID parent_id, String keyspaceName, String[] cfnames, RepairOption options) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 8f883f8f4783..0d30d409fe96 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -25,6 +25,8 @@ import com.google.common.base.Objects; import com.google.common.collect.ImmutableMap; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.Attributes; import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.exceptions.ConfigurationException; @@ -60,7 +62,8 @@ public enum Option ADDITIONAL_WRITE_POLICY, CRC_CHECK_CHANCE, CDC, - READ_REPAIR; + READ_REPAIR, + AUTO_REPAIR; @Override public String toString() @@ -88,6 +91,7 @@ public String toString() public final ImmutableMap extensions; public final boolean cdc; public final ReadRepairStrategy readRepair; + public final AutoRepairParams autoRepair; private TableParams(Builder builder) { @@ -112,6 +116,7 @@ private TableParams(Builder builder) extensions = builder.extensions; cdc = builder.cdc; readRepair = builder.readRepair; + autoRepair = builder.autoRepair; } public static Builder builder() @@ -139,7 +144,8 @@ public static Builder builder(TableParams params) .additionalWritePolicy(params.additionalWritePolicy) .extensions(params.extensions) .cdc(params.cdc) - .readRepair(params.readRepair); + .readRepair(params.readRepair) + .automatedRepair(params.autoRepair); } public Builder unbuild() @@ -153,7 +159,7 @@ public void validate() compression.validate(); double minBloomFilterFpChanceValue = BloomCalculations.minSupportedBloomFilterFpChance(); - if (bloomFilterFpChance <= minBloomFilterFpChanceValue || bloomFilterFpChance > 1) + if (bloomFilterFpChance <= minBloomFilterFpChanceValue || bloomFilterFpChance > 1) { fail("%s must be larger than %s and less than or equal to 1.0 (got %s)", BLOOM_FILTER_FP_CHANCE, @@ -194,6 +200,8 @@ public void validate() if (cdc && memtable.factory().writesShouldSkipCommitLog()) fail("CDC cannot work if writes skip the commit log. Check your memtable configuration."); + + autoRepair.validate(); } private static void fail(String format, Object... args) @@ -217,7 +225,7 @@ public boolean equals(Object o) && allowAutoSnapshot == p.allowAutoSnapshot && bloomFilterFpChance == p.bloomFilterFpChance && crcCheckChance == p.crcCheckChance - && gcGraceSeconds == p.gcGraceSeconds + && gcGraceSeconds == p.gcGraceSeconds && incrementalBackups == p.incrementalBackups && defaultTimeToLive == p.defaultTimeToLive && memtableFlushPeriodInMs == p.memtableFlushPeriodInMs @@ -230,7 +238,8 @@ public boolean equals(Object o) && memtable.equals(p.memtable) && extensions.equals(p.extensions) && cdc == p.cdc - && readRepair == p.readRepair; + && readRepair == p.readRepair + && autoRepair.equals(p.autoRepair); } @Override @@ -254,7 +263,8 @@ public int hashCode() memtable, extensions, cdc, - readRepair); + readRepair, + autoRepair); } @Override @@ -280,6 +290,7 @@ public String toString() .add(EXTENSIONS.toString(), extensions) .add(CDC.toString(), cdc) .add(READ_REPAIR.toString(), readRepair) + .add(AUTO_REPAIR.toString(), autoRepair) .toString(); } @@ -332,6 +343,14 @@ public void appendCqlTo(CqlBuilder builder, boolean isView) .append("AND read_repair = ").appendWithSingleQuotes(readRepair.toString()) .newLine() .append("AND speculative_retry = ").appendWithSingleQuotes(speculativeRetry.toString()); + + if (CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean() + && DatabaseDescriptor.getRawConfig() != null + && DatabaseDescriptor.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) + { + builder.newLine() + .append("AND auto_repair = ").append(autoRepair.asMap()); + } } public static final class Builder @@ -356,6 +375,7 @@ public static final class Builder private boolean cdc; private ReadRepairStrategy readRepair = ReadRepairStrategy.BLOCKING; + private AutoRepairParams autoRepair = AutoRepairParams.DEFAULT; public Builder() { } @@ -478,5 +498,11 @@ public Builder extensions(Map val) extensions = ImmutableMap.copyOf(val); return this; } + + public Builder automatedRepair(AutoRepairParams val) + { + autoRepair = val; + return this; + } } } diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index e120122c0844..5c8ca677c019 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -41,6 +41,9 @@ import com.google.common.collect.Lists; import com.google.common.collect.Multimap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DurationSpec; @@ -50,6 +53,7 @@ import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.EndpointsByRange; import org.apache.cassandra.locator.EndpointsForRange; +import org.apache.cassandra.service.disk.usage.DiskUsageMonitor; import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.repair.state.CoordinatorState; import org.apache.cassandra.repair.state.ParticipateState; @@ -61,8 +65,6 @@ import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.concurrent.AsyncPromise; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; @@ -645,8 +647,26 @@ public boolean verifyCompactionsPendingThreshold(TimeUUID parentRepairSession, P return true; } + public boolean verifyDiskHeadroomThreshold(TimeUUID parentRepairSession, PreviewKind previewKind) + { + double diskUsage = DiskUsageMonitor.instance.getDiskUsage(); + double rejectRatio = getIncrementalRepairDiskHeadroomRejectRatio(); + + if (diskUsage + rejectRatio > 1) + { + logger.error("[{}] Rejecting incoming repair, disk usage ({}%) above threshold ({}%)", + previewKind.logPrefix(parentRepairSession), String.format("%.2f", diskUsage * 100), String.format("%.2f", (1 - rejectRatio) * 100)); + return false; + } + + return true; + } + public Future prepareForRepair(TimeUUID parentRepairSession, InetAddressAndPort coordinator, Set endpoints, RepairOption options, boolean isForcedRepair, List columnFamilyStores) { + if (!verifyDiskHeadroomThreshold(parentRepairSession, options.getPreviewKind())) + failRepair(parentRepairSession, "Rejecting incoming repair, disk usage above threshold"); // failRepair throws exception + if (!verifyCompactionsPendingThreshold(parentRepairSession, options.getPreviewKind())) failRepair(parentRepairSession, "Rejecting incoming repair, pending compactions above threshold"); // failRepair throws exception @@ -1059,6 +1079,16 @@ public void setRepairPendingCompactionRejectThreshold(int value) DatabaseDescriptor.setRepairPendingCompactionRejectThreshold(value); } + public double getIncrementalRepairDiskHeadroomRejectRatio() + { + return DatabaseDescriptor.getRepairDiskHeadroomRejectRatio(); + } + + public void setIncrementalRepairDiskHeadroomRejectRatio(double value) + { + DatabaseDescriptor.setRepairDiskHeadroomRejectRatio(value); + } + /** * Remove any parent repair sessions matching predicate */ diff --git a/src/java/org/apache/cassandra/service/ActiveRepairServiceMBean.java b/src/java/org/apache/cassandra/service/ActiveRepairServiceMBean.java index 851dc6c802bb..c739b048d68f 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairServiceMBean.java @@ -74,4 +74,8 @@ public interface ActiveRepairServiceMBean int parentRepairSessionsCount(); public int getPaxosRepairParallelism(); public void setPaxosRepairParallelism(int v); + + public double getIncrementalRepairDiskHeadroomRejectRatio(); + + public void setIncrementalRepairDiskHeadroomRejectRatio(double value); } diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java new file mode 100644 index 000000000000..52cd9b394087 --- /dev/null +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -0,0 +1,349 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.service; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.ParameterizedClass; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.utils.MBeanWrapper; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implement all the MBeans for AutoRepair. + */ +public class AutoRepairService implements AutoRepairServiceMBean +{ + private static final Logger logger = LoggerFactory.getLogger(AutoRepairService.class); + + public static final String MBEAN_NAME = "org.apache.cassandra.db:type=AutoRepairService"; + + @VisibleForTesting + protected AutoRepairConfig config; + + public static final AutoRepairService instance = new AutoRepairService(); + + @VisibleForTesting + protected AutoRepairService() + { + } + + public static void setup() + { + if (!CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean()) + { + logger.info("Auto-repair service is disabled via JVM property"); + return; + } + instance.config = DatabaseDescriptor.getAutoRepairConfig(); + } + + static + { + if (CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean()) + { + MBeanWrapper.instance.registerMBean(instance, MBEAN_NAME); + } + } + + public void checkCanRun(String repairType) + { + checkCanRun(RepairType.parse(repairType)); + } + + public void checkCanRun(RepairType repairType) + { + if (!config.isAutoRepairSchedulingEnabled()) + throw new ConfigurationException("Auto-repair scheduler is disabled."); + + if (repairType != RepairType.INCREMENTAL) + return; + + if (config.getMaterializedViewRepairEnabled(repairType) && DatabaseDescriptor.isMaterializedViewsOnRepairEnabled()) + throw new ConfigurationException("Cannot run incremental repair while materialized view replay is enabled. Set materialized_views_on_repair_enabled to false."); + + if (DatabaseDescriptor.isCDCEnabled() && DatabaseDescriptor.isCDCOnRepairEnabled()) + throw new ConfigurationException("Cannot run incremental repair while CDC replay is enabled. Set cdc_on_repair_enabled to false."); + } + + public AutoRepairConfig getAutoRepairConfig() + { + return config; + } + + @Override + public boolean isAutoRepairDisabled() + { + return !CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean() + || config == null + || !config.isAutoRepairSchedulingEnabled(); + } + + @Override + public String getAutoRepairConfiguration() + { + StringBuilder sb = new StringBuilder(); + sb.append("repair scheduler configuration:"); + appendConfig(sb, "repair_check_interval", config.getRepairCheckInterval()); + appendConfig(sb, "repair_task_min_duration", config.getRepairTaskMinDuration()); + appendConfig(sb, "history_clear_delete_hosts_buffer_interval", config.getAutoRepairHistoryClearDeleteHostsBufferInterval()); + appendConfig(sb, "mixed_major_version_repair_enabled", config.getMixedMajorVersionRepairEnabled()); + for (RepairType repairType : RepairType.values()) + { + sb.append(formatRepairTypeConfig(repairType, config)); + } + return sb.toString(); + } + + @Override + public void setAutoRepairEnabled(String repairType, boolean enabled) + { + checkCanRun(repairType); + config.setAutoRepairEnabled(RepairType.parse(repairType), enabled); + } + + @Override + public void setRepairThreads(String repairType, int repairThreads) + { + config.setRepairThreads(RepairType.parse(repairType), repairThreads); + } + + @Override + public void setRepairPriorityForHosts(String repairType, String commaSeparatedHostSet) + { + Set hosts = InetAddressAndPort.parseHosts(commaSeparatedHostSet, false); + if (!hosts.isEmpty()) + { + AutoRepairUtils.addPriorityHosts(RepairType.parse(repairType), hosts); + } + } + + @Override + public void setForceRepairForHosts(String repairType, String commaSeparatedHostSet) + { + Set hosts = InetAddressAndPort.parseHosts(commaSeparatedHostSet, false); + if (!hosts.isEmpty()) + { + AutoRepairUtils.setForceRepair(RepairType.parse(repairType), hosts); + } + } + + @Override + public void setRepairMinInterval(String repairType, String minRepairInterval) + { + config.setRepairMinInterval(RepairType.parse(repairType), minRepairInterval); + } + + @Override + public void startScheduler() + { + config.startScheduler(); + } + + @Override + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) + { + config.setAutoRepairHistoryClearDeleteHostsBufferInterval(duration); + } + + @Override + public void setAutoRepairMinRepairTaskDuration(String duration) + { + config.setRepairTaskMinDuration(duration); + } + + @Override + public void setRepairSSTableCountHigherThreshold(String repairType, int sstableHigherThreshold) + { + config.setRepairSSTableCountHigherThreshold(RepairType.parse(repairType), sstableHigherThreshold); + } + + @Override + public void setAutoRepairTableMaxRepairTime(String repairType, String autoRepairTableMaxRepairTime) + { + config.setAutoRepairTableMaxRepairTime(RepairType.parse(repairType), autoRepairTableMaxRepairTime); + } + + @Override + public void setIgnoreDCs(String repairType, Set ignoreDCs) + { + config.setIgnoreDCs(RepairType.parse(repairType), ignoreDCs); + } + + @Override + public void setPrimaryTokenRangeOnly(String repairType, boolean primaryTokenRangeOnly) + { + config.setRepairPrimaryTokenRangeOnly(RepairType.parse(repairType), primaryTokenRangeOnly); + } + + @Override + public void setParallelRepairPercentage(String repairType, int percentage) + { + config.setParallelRepairPercentage(RepairType.parse(repairType), percentage); + } + + @Override + public void setParallelRepairCount(String repairType, int count) + { + config.setParallelRepairCount(RepairType.parse(repairType), count); + } + + @Override + public void setAllowParallelReplicaRepair(String repairType, boolean enabled) + { + config.setAllowParallelReplicaRepair(RepairType.parse(repairType), enabled); + } + + @Override + public void setAllowParallelReplicaRepairAcrossSchedules(String repairType, boolean enabled) + { + config.setAllowParallelReplicaRepairAcrossSchedules(RepairType.parse(repairType), enabled); + } + + @Override + public void setMVRepairEnabled(String repairType, boolean enabled) + { + config.setMaterializedViewRepairEnabled(RepairType.parse(repairType), enabled); + } + + @Override + public void setRepairSessionTimeout(String repairType, String timeout) + { + config.setRepairSessionTimeout(RepairType.parse(repairType), timeout); + } + + @Override + public Set getOnGoingRepairHostIds(String repairType) + { + List histories = AutoRepairUtils.getAutoRepairHistory(RepairType.parse(repairType)); + if (histories == null) + { + return Collections.emptySet(); + } + Set hostIds = new HashSet<>(); + AutoRepairUtils.CurrentRepairStatus currentRepairStatus = new AutoRepairUtils.CurrentRepairStatus(histories, AutoRepairUtils.getPriorityHostIds(RepairType.parse(repairType)), null); + for (UUID id : currentRepairStatus.hostIdsWithOnGoingRepair) + { + hostIds.add(id.toString()); + } + for (UUID id : currentRepairStatus.hostIdsWithOnGoingForceRepair) + { + hostIds.add(id.toString()); + } + return Collections.unmodifiableSet(hostIds); + } + + @Override + public void setAutoRepairTokenRangeSplitterParameter(String repairType, String key, String value) + { + config.getTokenRangeSplitterInstance(RepairType.parse(repairType)).setParameter(key, value); + } + + @Override + public void setRepairByKeyspace(String repairType, boolean repairByKeyspace) + { + config.setRepairByKeyspace(RepairType.parse(repairType), repairByKeyspace); + } + + @Override + public void setAutoRepairMaxRetriesCount(String repairType, int retries) + { + config.setRepairMaxRetries(RepairType.parse(repairType), retries); + } + + @Override + public void setAutoRepairRetryBackoff(String repairType, String interval) + { + config.setRepairRetryBackoff(RepairType.parse(repairType), interval); + } + + @Override + public void setMixedMajorVersionRepairEnabled(boolean enabled) + { + config.setMixedMajorVersionRepairEnabled(enabled); + } + + private String formatRepairTypeConfig(RepairType repairType, AutoRepairConfig config) + { + StringBuilder sb = new StringBuilder(); + sb.append("\nconfiguration for repair_type: ").append(repairType.getConfigName()); + sb.append("\n\tenabled: ").append(config.isAutoRepairEnabled(repairType)); + // Only show configuration if enabled + if (config.isAutoRepairEnabled(repairType)) + { + Set priorityHosts = AutoRepairUtils.getPriorityHosts(repairType); + if (!priorityHosts.isEmpty()) + { + appendConfig(sb, "priority_hosts", Joiner.on(',').skipNulls().join(priorityHosts)); + } + + appendConfig(sb, "min_repair_interval", config.getRepairMinInterval(repairType)); + appendConfig(sb, "repair_by_keyspace", config.getRepairByKeyspace(repairType)); + appendConfig(sb, "number_of_repair_threads", config.getRepairThreads(repairType)); + appendConfig(sb, "sstable_upper_threshold", config.getRepairSSTableCountHigherThreshold(repairType)); + appendConfig(sb, "table_max_repair_time", config.getAutoRepairTableMaxRepairTime(repairType)); + appendConfig(sb, "ignore_dcs", config.getIgnoreDCs(repairType)); + appendConfig(sb, "repair_primary_token_range_only", config.getRepairPrimaryTokenRangeOnly(repairType)); + appendConfig(sb, "parallel_repair_count", config.getParallelRepairCount(repairType)); + appendConfig(sb, "parallel_repair_percentage", config.getParallelRepairPercentage(repairType)); + appendConfig(sb, "allow_parallel_replica_repair", config.getAllowParallelReplicaRepair(repairType)); + appendConfig(sb, "allow_parallel_replica_repair_across_schedules", config.getAllowParallelReplicaRepairAcrossSchedules(repairType)); + appendConfig(sb, "materialized_view_repair_enabled", config.getMaterializedViewRepairEnabled(repairType)); + appendConfig(sb, "initial_scheduler_delay", config.getInitialSchedulerDelay(repairType)); + appendConfig(sb, "repair_session_timeout", config.getRepairSessionTimeout(repairType)); + appendConfig(sb, "force_repair_new_node", config.getForceRepairNewNode(repairType)); + appendConfig(sb, "repair_max_retries", config.getRepairMaxRetries(repairType)); + appendConfig(sb, "repair_retry_backoff", config.getRepairRetryBackoff(repairType)); + + final ParameterizedClass splitterClass = config.getTokenRangeSplitter(repairType); + final String splitterClassName = splitterClass.class_name != null ? splitterClass.class_name : AutoRepairConfig.DEFAULT_SPLITTER.getName(); + appendConfig(sb, "token_range_splitter", splitterClassName); + Map tokenRangeSplitterParameters = config.getTokenRangeSplitterInstance(repairType).getParameters(); + if (!tokenRangeSplitterParameters.isEmpty()) + { + for (Map.Entry param : tokenRangeSplitterParameters.entrySet()) + { + appendConfig(sb, String.format("token_range_splitter.%s", param.getKey()), param.getValue()); + } + } + } + + return sb.toString(); + } + + private void appendConfig(StringBuilder sb, String config, T value) + { + sb.append(String.format("%s%s: %s", "\n\t", config, value)); + } +} diff --git a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java new file mode 100644 index 000000000000..e4d554dd980d --- /dev/null +++ b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.service; + + +import java.util.Set; + +/** + * Defines all the MBeans exposed for AutoRepair. + */ +public interface AutoRepairServiceMBean +{ + public void setAutoRepairEnabled(String repairType, boolean enabled); + + public void setRepairThreads(String repairType, int repairThreads); + + public void setRepairPriorityForHosts(String repairType, String commaSeparatedHostSet); + + public void setForceRepairForHosts(String repairType, String commaSeparatedHostSet); + + public void setRepairMinInterval(String repairType, String minRepairInterval); + + void startScheduler(); + + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration); + + public void setAutoRepairMinRepairTaskDuration(String duration); + + public void setRepairSSTableCountHigherThreshold(String repairType, int ssTableHigherThreshold); + + public void setAutoRepairTableMaxRepairTime(String repairType, String autoRepairTableMaxRepairTime); + + public void setIgnoreDCs(String repairType, Set ignorDCs); + + public void setPrimaryTokenRangeOnly(String repairType, boolean primaryTokenRangeOnly); + + public void setParallelRepairPercentage(String repairType, int percentage); + + public void setParallelRepairCount(String repairType, int count); + + public void setAllowParallelReplicaRepair(String repairType, boolean enabled); + + public void setAllowParallelReplicaRepairAcrossSchedules(String repairType, boolean enabled); + + public void setMVRepairEnabled(String repairType, boolean enabled); + + public boolean isAutoRepairDisabled(); + + public String getAutoRepairConfiguration(); + + public void setRepairSessionTimeout(String repairType, String timeout); + + public Set getOnGoingRepairHostIds(String repairType); + + public void setAutoRepairTokenRangeSplitterParameter(String repairType, String key, String value); + + public void setRepairByKeyspace(String repairType, boolean repairByKeyspace); + + public void setAutoRepairMaxRetriesCount(String repairType, int retries); + + public void setAutoRepairRetryBackoff(String repairType, String interval); + + public void setMixedMajorVersionRepairEnabled(boolean enabled); +} diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index a284fa27386e..7dd68e3224c4 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -88,6 +88,10 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.RateLimiter; import com.google.common.util.concurrent.Uninterruptibles; + +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -183,6 +187,7 @@ import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.repair.SharedContext; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.repair.messages.RepairOption; import org.apache.cassandra.schema.CompactionParams.TombstoneOption; import org.apache.cassandra.schema.KeyspaceMetadata; @@ -267,6 +272,7 @@ import static org.apache.cassandra.index.SecondaryIndexManager.isIndexColumnFamily; import static org.apache.cassandra.net.NoPayload.noPayload; import static org.apache.cassandra.net.Verb.REPLICATION_DONE_REQ; +import static org.apache.cassandra.locator.InetAddressAndPort.stringify; import static org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus; import static org.apache.cassandra.service.ActiveRepairService.repairCommandExecutor; import static org.apache.cassandra.service.StorageService.Mode.DECOMMISSIONED; @@ -408,7 +414,12 @@ public RangesAtEndpoint getLocalReplicas(String keyspaceName) public RangesAtEndpoint getReplicas(String keyspaceName, InetAddressAndPort endpoint) { - return Keyspace.open(keyspaceName).getReplicationStrategy().getAddressReplicas(endpoint); + return getReplicas(Keyspace.open(keyspaceName).getReplicationStrategy(), endpoint); + } + + public RangesAtEndpoint getReplicas(AbstractReplicationStrategy replicationStrategy, InetAddressAndPort endpoint) + { + return replicationStrategy.getAddressReplicas(tokenMetadata.cloneOnlyTokenMap(), endpoint); } public List> getLocalRanges(String ks) @@ -482,7 +493,7 @@ public enum Mode { STARTING, NORMAL, JOINING, JOINING_FAILED, LEAVING, DECOMMISS /* Used for tracking drain progress */ private volatile int totalCFs, remainingCFs; - private static final AtomicInteger nextRepairCommand = new AtomicInteger(); + public static final AtomicInteger nextRepairCommand = new AtomicInteger(); private final List lifecycleSubscribers = new CopyOnWriteArrayList<>(); @@ -1313,6 +1324,15 @@ public void joinTokenRing(boolean finishJoiningRing, if (dataAvailable) { finishJoiningRing(shouldBootstrap, bootstrapTokens); + AutoRepairConfig repairConfig = DatabaseDescriptor.getAutoRepairConfig(); + // this node might have just bootstrapped; check if we should run repair immediately + if (shouldBootstrap && CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean() + && repairConfig.isAutoRepairSchedulingEnabled()) + { + for (AutoRepairConfig.RepairType rType : AutoRepairConfig.RepairType.values()) + if (repairConfig.isAutoRepairEnabled(rType) && repairConfig.getForceRepairNewNode(rType)) + AutoRepairUtils.setForceRepairNewNode(rType); + } // remove the existing info about the replaced node. if (!current.isEmpty()) { @@ -1415,6 +1435,8 @@ public void finishJoiningRing(boolean didBootstrap, Collection tokens) setTokens(tokens); assert tokenMetadata.sortedTokens().size() > 0; + + doAutoRepairSetup(); } @VisibleForTesting @@ -1438,6 +1460,24 @@ public void doAuthSetup(boolean setUpSchema) } } + public void doAutoRepairSetup() + { + if (!CassandraRelevantProperties.AUTOREPAIR_ENABLE.getBoolean()) + { + logger.info("Auto-repair service is disabled via JVM property, skipping setup"); + return; + } + + AutoRepairService.setup(); + if (DatabaseDescriptor.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) + { + logger.info("Enabling auto-repair scheduling"); + AutoRepair.instance.setup(); + logger.info("AutoRepair setup complete!"); + } + } + + public boolean isAuthSetupComplete() { return authSetupComplete; @@ -4102,16 +4142,6 @@ public String getSavedCachesLocation() return FileUtils.getCanonicalPath(DatabaseDescriptor.getSavedCachesLocation()); } - private List stringify(Iterable endpoints, boolean withPort) - { - List stringEndpoints = new ArrayList<>(); - for (InetAddressAndPort ep : endpoints) - { - stringEndpoints.add(ep.getHostAddress(withPort)); - } - return stringEndpoints; - } - public int getCurrentGenerationNumber() { return Gossiper.instance.getCurrentGenerationNumber(FBUtilities.getBroadcastAddressAndPort()); @@ -5935,6 +5965,8 @@ protected synchronized void drain(boolean isFinalShutdown) throws IOException, I CommitLog.instance.shutdownBlocking(); + AutoRepair.instance.shutdownBlocking(); + // wait for miscellaneous tasks like sstable and commitlog segment deletion ColumnFamilyStore.shutdownPostFlushExecutor(); @@ -7686,4 +7718,41 @@ public boolean getPaxosRepairRaceWait() { return DatabaseDescriptor.getPaxosRepairRaceWait(); } + + @Override + public List getTablesForKeyspace(String keyspace) + { + return Keyspace.open(keyspace).getColumnFamilyStores().stream().map(cfs -> cfs.name).collect(Collectors.toList()); + } + + @Override + public List mutateSSTableRepairedState(boolean repaired, boolean preview, String keyspace, List tableNames) + { + Map tables = Keyspace.open(keyspace).getColumnFamilyStores() + .stream().collect(Collectors.toMap(c -> c.name, c -> c)); + for (String tableName : tableNames) + { + if (!tables.containsKey(tableName)) + throw new RuntimeException("Table " + tableName + " does not exist in keyspace " + keyspace); + } + + // only select SSTables that are unrepaired when repaired is true and vice versa + Predicate predicate = sst -> repaired != sst.isRepaired(); + + // mutate SSTables + long repairedAt = !repaired ? 0 : currentTimeMillis(); + List sstablesTouched = new ArrayList<>(); + for (String tableName : tableNames) + { + ColumnFamilyStore table = tables.get(tableName); + Set result = table.runWithCompactionsDisabled(() -> { + Set sstables = table.getLiveSSTables().stream().filter(predicate).collect(Collectors.toSet()); + if (!preview) + table.getCompactionStrategyManager().mutateRepaired(sstables, repairedAt, null, false); + return sstables; + }, predicate, OperationType.ANTICOMPACTION, true, false, true); + sstablesTouched.addAll(result.stream().map(sst -> sst.descriptor.baseFile().name()).collect(Collectors.toList())); + } + return sstablesTouched; + } } diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 1aec3d5280d3..ab07eec47e36 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -1328,4 +1328,10 @@ public void enableAuditLog(String loggerName, String includedKeyspaces, String e boolean getPaxosRepairRaceWait(); public void dropPreparedStatements(boolean memoryOnly); + + /** Gets the names of all tables for the given keyspace */ + public List getTablesForKeyspace(String keyspace); + + /** Mutates the repaired state of all SSTables for the given SSTables */ + public List mutateSSTableRepairedState(boolean repaired, boolean preview, String keyspace, List tables); } diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 0f53ccd4df66..c2492fbc1407 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -105,6 +105,8 @@ import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.MessagingServiceMBean; import org.apache.cassandra.service.ActiveRepairServiceMBean; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.AutoRepairServiceMBean; import org.apache.cassandra.service.CacheService; import org.apache.cassandra.service.CacheServiceMBean; import org.apache.cassandra.service.GCInspector; @@ -173,6 +175,7 @@ public class NodeProbe implements AutoCloseable protected PermissionsCacheMBean pcProxy; protected RolesCacheMBean rcProxy; protected GuardrailsMBean grProxy; + protected AutoRepairServiceMBean autoRepairProxy; protected Output output; private boolean failed; @@ -314,6 +317,8 @@ protected void connect() throws IOException name = new ObjectName(Guardrails.MBEAN_NAME); grProxy = JMX.newMBeanProxy(mbeanServerConn, name, GuardrailsMBean.class); + name = new ObjectName(AutoRepairService.MBEAN_NAME); + autoRepairProxy = JMX.newMBeanProxy(mbeanServerConn, name, AutoRepairServiceMBean.class); } catch (MalformedObjectNameException e) { @@ -2443,6 +2448,146 @@ public GuardrailsMBean getGuardrailsMBean() { return grProxy; } + + public boolean isAutoRepairDisabled() + { + return autoRepairProxy.isAutoRepairDisabled(); + } + + public String autoRepairConfiguration() + { + return autoRepairProxy.getAutoRepairConfiguration(); + } + + public void setAutoRepairTokenRangeSplitterParameter(String repairType, String key, String value) + { + autoRepairProxy.setAutoRepairTokenRangeSplitterParameter(repairType, key, value); + } + + public void setAutoRepairEnabled(String repairType, boolean enabled) + { + autoRepairProxy.setAutoRepairEnabled(repairType, enabled); + } + + public void setAutoRepairThreads(String repairType, int repairThreads) + { + autoRepairProxy.setRepairThreads(repairType, repairThreads); + } + + public void setAutoRepairPriorityForHosts(String repairType, String commaSeparatedHostSet) + { + autoRepairProxy.setRepairPriorityForHosts(repairType, commaSeparatedHostSet); + } + + public void setAutoRepairForceRepairForHosts(String repairType, String commaSeparatedHostSet) + { + autoRepairProxy.setForceRepairForHosts(repairType, commaSeparatedHostSet); + } + + public void setAutoRepairMinInterval(String repairType, String minRepairInterval) + { + autoRepairProxy.setRepairMinInterval(repairType, minRepairInterval); + } + + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) + { + autoRepairProxy.setAutoRepairHistoryClearDeleteHostsBufferDuration(duration); + } + + public void startAutoRepairScheduler() + { + autoRepairProxy.startScheduler(); + } + + public void setAutoRepairMinRepairTaskDuration(String duration) + { + autoRepairProxy.setAutoRepairMinRepairTaskDuration(duration); + } + + public void setAutoRepairSSTableCountHigherThreshold(String repairType, int ssTableHigherThreshold) + { + autoRepairProxy.setRepairSSTableCountHigherThreshold(repairType, ssTableHigherThreshold); + } + + public void setAutoRepairTableMaxRepairTime(String repairType, String autoRepairTableMaxRepairTime) + { + autoRepairProxy.setAutoRepairTableMaxRepairTime(repairType, autoRepairTableMaxRepairTime); + } + + public void setAutoRepairIgnoreDCs(String repairType, Set ignoreDCs) + { + autoRepairProxy.setIgnoreDCs(repairType, ignoreDCs); + } + + public void setAutoRepairParallelRepairPercentage(String repairType, int percentage) + { + autoRepairProxy.setParallelRepairPercentage(repairType, percentage); + } + + public void setAutoRepairParallelRepairCount(String repairType, int count) + { + autoRepairProxy.setParallelRepairCount(repairType, count); + } + + public void setAutoRepairAllowParallelReplicaRepair(String repairType, boolean enabled) + { + autoRepairProxy.setAllowParallelReplicaRepair(repairType, enabled); + } + + public void setAutoRepairAllowParallelReplicaRepairAcrossSchedules(String repairType, boolean enabled) + { + autoRepairProxy.setAllowParallelReplicaRepairAcrossSchedules(repairType, enabled); + } + + public void setAutoRepairPrimaryTokenRangeOnly(String repairType, boolean primaryTokenRangeOnly) + { + autoRepairProxy.setPrimaryTokenRangeOnly(repairType, primaryTokenRangeOnly); + } + + public void setAutoRepairMaterializedViewRepairEnabled(String repairType, boolean enabled) + { + autoRepairProxy.setMVRepairEnabled(repairType, enabled); + } + + public List mutateSSTableRepairedState(boolean repair, boolean preview, String keyspace, List tables) + { + return ssProxy.mutateSSTableRepairedState(repair, preview, keyspace, tables); + } + + public List getAutoRepairTablesForKeyspace(String keyspace) + { + return ssProxy.getTablesForKeyspace(keyspace); + } + + public void setAutoRepairSessionTimeout(String repairType, String timeout) + { + autoRepairProxy.setRepairSessionTimeout(repairType, timeout); + } + + public Set getAutoRepairOnGoingRepairHostIds(String repairType) + { + return autoRepairProxy.getOnGoingRepairHostIds(repairType); + } + + public void setAutoRepairRepairByKeyspace(String repairType, boolean enabled) + { + autoRepairProxy.setRepairByKeyspace(repairType, enabled); + } + + public void setAutoRepairMaxRetriesCount(String repairType, int retries) + { + autoRepairProxy.setAutoRepairMaxRetriesCount(repairType, retries); + } + + public void setAutoRepairRetryBackoff(String repairType, String interval) + { + autoRepairProxy.setAutoRepairRetryBackoff(repairType, interval); + } + + public void setMixedMajorVersionRepairEnabled(boolean enabled) + { + autoRepairProxy.setMixedMajorVersionRepairEnabled(enabled); + } } class ColumnFamilyStoreMBeanIterator implements Iterator> diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index aa66719474a2..019319fcbcb9 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -96,6 +96,7 @@ public NodeTool(INodeProbeFactory nodeProbeFactory, Output output) public int execute(String... args) { List> commands = newArrayList( + AutoRepairStatus.class, Assassinate.class, CassHelp.class, CIDRFilteringStats.class, @@ -136,6 +137,7 @@ public int execute(String... args) GcStats.class, GetAuditLog.class, GetAuthCacheConfig.class, + GetAutoRepairConfig.class, GetBatchlogReplayTrottle.class, GetCIDRGroupsOfIP.class, GetColumnIndexSize.class, @@ -199,6 +201,7 @@ public int execute(String... args) Ring.class, Scrub.class, SetAuthCacheConfig.class, + SetAutoRepairConfig.class, SetBatchlogReplayThrottle.class, SetCacheCapacity.class, SetCacheKeysToSave.class, @@ -220,6 +223,7 @@ public int execute(String... args) SetTraceProbability.class, Sjk.class, Snapshot.class, + SSTableRepairedSet.class, Status.class, StatusAutoCompaction.class, StatusBackup.class, diff --git a/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java new file mode 100644 index 000000000000..bb594a010ff1 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.tools.nodetool; + +import java.io.PrintStream; +import java.util.Set; + +import com.google.common.annotations.VisibleForTesting; + +import io.airlift.airline.Command; +import io.airlift.airline.Option; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool; +import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * Provides currently running auto-repair tasks. + */ +@Command(name = "autorepairstatus", description = "Print autorepair status") +public class AutoRepairStatus extends NodeTool.NodeToolCmd +{ + @VisibleForTesting + @Option(title = "repair type", name = { "-t", "--repair-type" }, description = "Repair type") + protected String repairType; + + @Override + public void execute(NodeProbe probe) + { + checkArgument(repairType != null, "--repair-type is required."); + PrintStream out = probe.output().out; + + if (probe.isAutoRepairDisabled()) + { + out.println("Auto-repair is not enabled"); + return; + } + + TableBuilder table = new TableBuilder(); + table.add("Active Repairs"); + Set ongoingRepairHostIds = probe.getAutoRepairOnGoingRepairHostIds(repairType); + table.add(getSetString(ongoingRepairHostIds)); + table.printTo(out); + } + + private String getSetString(Set hostIds) + { + if (hostIds.isEmpty()) + { + return "NONE"; + } + StringBuilder sb = new StringBuilder(); + for (String id : hostIds) + { + sb.append(id); + sb.append(","); + } + // remove last "," + sb.setLength(Math.max(sb.length() - 1, 0)); + return sb.toString(); + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java new file mode 100644 index 000000000000..9744498de757 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.tools.nodetool; + +import java.io.PrintStream; + +import com.google.common.annotations.VisibleForTesting; + +import io.airlift.airline.Command; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool.NodeToolCmd; + +/** + * Prints all the configurations for AutoRepair through nodetool. + */ +@Command(name = "getautorepairconfig", description = "Print autorepair configurations") +public class GetAutoRepairConfig extends NodeToolCmd +{ + @VisibleForTesting + protected static PrintStream out = System.out; + + @Override + public void execute(NodeProbe probe) + { + if (probe.isAutoRepairDisabled()) + out.println("Auto-repair is not enabled"); + else + out.println(probe.autoRepairConfiguration()); + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java b/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java new file mode 100644 index 000000000000..2a7b56732ac9 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.tools.nodetool; + +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + + +import io.airlift.airline.Arguments; +import io.airlift.airline.Command; +import io.airlift.airline.Option; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool; + +/** + * Provides a way to set the repaired state of SSTables without any downtime through nodetool. + */ +@Command(name = "sstablerepairedset", description = "Set the repaired state of SSTables for given keyspace/tables") +public class SSTableRepairedSet extends NodeTool.NodeToolCmd +{ + @Arguments(usage = "[ ]", description = "Optional keyspace followed by zero or more tables") + protected List args = new ArrayList<>(); + + @Option(title = "really-set", + name = { "--really-set" }, + description = "Really set the repaired state of SSTables. If not set, only print SSTables that would be affected.") + protected boolean reallySet = false; + + @Option(title = "is-repaired", + name = { "--is-repaired" }, + description = "Set SSTables to repaired state.") + protected boolean isRepaired = false; + + @Option(title = "is-unrepaired", + name = { "--is-unrepaired" }, + description = "Set SSTables to unrepaired state.") + protected boolean isUnrepaired = false; + + @Override + public void execute(NodeProbe probe) + { + PrintStream out = probe.output().out; + + if (isRepaired == isUnrepaired) + { + out.println("Exactly one of --is-repaired or --is-unrepaired must be provided."); + return; + } + + String message; + if (reallySet) + message = "Mutating repaired state of SSTables for"; + else + message = "Previewing repaired state mutation of SSTables for"; + + List keyspaces = parseOptionalKeyspace(args, probe, KeyspaceSet.NON_LOCAL_STRATEGY); + List tables = new ArrayList<>(Arrays.asList(parseOptionalTables(args))); + + if (args.isEmpty()) + message += " all keyspaces"; + else + message += tables.isEmpty() ? " all tables" : " tables " + String.join(", ", tables) + + " in keyspace " + keyspaces.get(0); + message += " to " + (isRepaired ? "repaired" : "unrepaired"); + out.println(message); + + List sstableList = new ArrayList<>(); + for (String keyspace : keyspaces) + { + try + { + sstableList.addAll(probe.mutateSSTableRepairedState(isRepaired, !reallySet, keyspace, + tables.isEmpty() + ? probe.getAutoRepairTablesForKeyspace(keyspace) // mutate all tables + : tables)); // mutate specific tables + } + catch (InvalidRequestException e) + { + out.println(e.getMessage()); + } + } + if (!reallySet) + out.println("The following SSTables would be mutated:"); + else + out.println("The following SSTables were mutated:"); + for (String sstable : sstableList) + out.println(sstable); + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java new file mode 100644 index 000000000000..376023c2d5f6 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.tools.nodetool; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Splitter; + +import io.airlift.airline.Arguments; +import io.airlift.airline.Command; +import io.airlift.airline.Option; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool.NodeToolCmd; + +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * Allows to set AutoRepair configuration through nodetool. + */ +@Command(name = "setautorepairconfig", description = "sets the autorepair configuration") +public class SetAutoRepairConfig extends NodeToolCmd +{ + @VisibleForTesting + @Arguments(title = " ", usage = " ", + description = "autorepair param and value.\nPossible autorepair parameters are as following: " + + "[start_scheduler|number_of_repair_threads|min_repair_interval|sstable_upper_threshold" + + "|enabled|table_max_repair_time|priority_hosts|forcerepair_hosts|ignore_dcs" + + "|history_clear_delete_hosts_buffer_interval|repair_primary_token_range_only" + + "|parallel_repair_count|parallel_repair_percentage" + + "|allow_parallel_replica_repair|allow_parallel_repair_across_schedules" + + "|materialized_view_repair_enabled|repair_max_retries" + + "|repair_retry_backoff|repair_session_timeout|min_repair_task_duration" + + "|repair_by_keyspace|mixed_major_version_repair_enabled|token_range_splitter.]", + required = true) + protected List args = new ArrayList<>(); + + @VisibleForTesting + @Option(title = "repair type", name = { "-t", "--repair-type" }, description = "Repair type") + protected String repairTypeStr; + + @VisibleForTesting + protected PrintStream out = System.out; + + private static final String TOKEN_RANGE_SPLITTER_PROPERTY_PREFIX = "token_range_splitter."; + + @Override + public void execute(NodeProbe probe) + { + checkArgument(args.size() == 2, "setautorepairconfig requires param-type, and value args."); + String paramType = args.get(0); + String paramVal = args.get(1); + + if (probe.isAutoRepairDisabled() && !paramType.equalsIgnoreCase("start_scheduler")) + { + out.println("Auto-repair is not enabled"); + return; + } + + // options that do not require --repair-type option + switch (paramType) + { + case "start_scheduler": + if (Boolean.parseBoolean(paramVal)) + { + probe.startAutoRepairScheduler(); + } + return; + case "history_clear_delete_hosts_buffer_interval": + probe.setAutoRepairHistoryClearDeleteHostsBufferDuration(paramVal); + return; + case "min_repair_task_duration": + probe.setAutoRepairMinRepairTaskDuration(paramVal); + return; + case "mixed_major_version_repair_enabled": + probe.setMixedMajorVersionRepairEnabled(Boolean.parseBoolean(paramVal)); + return; + default: + // proceed to options that require --repair-type option + break; + } + + // options below require --repair-type option + Objects.requireNonNull(repairTypeStr, "--repair-type is required for this parameter."); + + if(paramType.startsWith(TOKEN_RANGE_SPLITTER_PROPERTY_PREFIX)) + { + final String key = paramType.replace(TOKEN_RANGE_SPLITTER_PROPERTY_PREFIX, ""); + probe.setAutoRepairTokenRangeSplitterParameter(repairTypeStr, key, paramVal); + return; + } + + switch (paramType) + { + case "enabled": + probe.setAutoRepairEnabled(repairTypeStr, Boolean.parseBoolean(paramVal)); + break; + case "number_of_repair_threads": + probe.setAutoRepairThreads(repairTypeStr, Integer.parseInt(paramVal)); + break; + case "min_repair_interval": + probe.setAutoRepairMinInterval(repairTypeStr, paramVal); + break; + case "sstable_upper_threshold": + probe.setAutoRepairSSTableCountHigherThreshold(repairTypeStr, Integer.parseInt(paramVal)); + break; + case "table_max_repair_time": + probe.setAutoRepairTableMaxRepairTime(repairTypeStr, paramVal); + break; + case "priority_hosts": + if (paramVal!= null && !paramVal.isEmpty()) + { + probe.setAutoRepairPriorityForHosts(repairTypeStr, paramVal); + } + break; + case "forcerepair_hosts": + probe.setAutoRepairForceRepairForHosts(repairTypeStr, paramVal); + break; + case "ignore_dcs": + Set ignoreDCs = new HashSet<>(); + for (String dc : Splitter.on(',').split(paramVal)) + { + ignoreDCs.add(dc); + } + probe.setAutoRepairIgnoreDCs(repairTypeStr, ignoreDCs); + break; + case "repair_primary_token_range_only": + probe.setAutoRepairPrimaryTokenRangeOnly(repairTypeStr, Boolean.parseBoolean(paramVal)); + break; + case "parallel_repair_count": + probe.setAutoRepairParallelRepairCount(repairTypeStr, Integer.parseInt(paramVal)); + break; + case "parallel_repair_percentage": + probe.setAutoRepairParallelRepairPercentage(repairTypeStr, Integer.parseInt(paramVal)); + break; + case "allow_parallel_replica_repair": + probe.setAutoRepairAllowParallelReplicaRepair(repairTypeStr, Boolean.parseBoolean(paramVal)); + break; + case "allow_parallel_replica_repair_across_schedules": + probe.setAutoRepairAllowParallelReplicaRepairAcrossSchedules(repairTypeStr, Boolean.parseBoolean(paramVal)); + break; + case "materialized_view_repair_enabled": + probe.setAutoRepairMaterializedViewRepairEnabled(repairTypeStr, Boolean.parseBoolean(paramVal)); + break; + case "repair_session_timeout": + probe.setAutoRepairSessionTimeout(repairTypeStr, paramVal); + break; + case "repair_by_keyspace": + probe.setAutoRepairRepairByKeyspace(repairTypeStr, Boolean.parseBoolean(paramVal)); + break; + case "repair_max_retries": + probe.setAutoRepairMaxRetriesCount(repairTypeStr, Integer.parseInt(paramVal)); + break; + case "repair_retry_backoff": + probe.setAutoRepairRetryBackoff(repairTypeStr, paramVal); + break; + default: + throw new IllegalArgumentException("Unknown parameter: " + paramType); + } + } +} diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index ca3444d4654a..f820dafe4393 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -68,6 +68,7 @@ import com.google.common.collect.ImmutableList; import com.vdurmont.semver4j.Semver; import com.vdurmont.semver4j.SemverException; + import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairFlagToggleTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairFlagToggleTest.java new file mode 100644 index 000000000000..72db50a4131b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairFlagToggleTest.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.repair; + +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.test.TestBaseImpl; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests enabling the AUTOREPAIR_ENABLE flag after initial deployment. + *

    + * This test verifies that: + * 1. When started with AUTOREPAIR_ENABLE=false, the auto_repair column does not appear in DESCRIBE TABLE + * 2. After restart with AUTOREPAIR_ENABLE=true, the auto_repair column appears in DESCRIBE TABLE + * 3. The system_distributed auto-repair tables are created after enabling + *

    + * Note: This is the supported upgrade path. The reverse (enabled → disabled) is NOT supported + * and will cause the node to fail during initialization due to schema incompatibility. + */ +public class AutoRepairFlagToggleTest extends TestBaseImpl +{ + @Test + public void testEnablingAutoRepairFlag() throws Exception + { + // Phase 1: Start with AUTOREPAIR_ENABLE=false + CassandraRelevantProperties.AUTOREPAIR_ENABLE.setBoolean(false); + + try (Cluster cluster = Cluster.build(1) + .withConfig(config -> config + .with(Feature.GOSSIP, Feature.NETWORK)) + .start()) + { + // Create a test keyspace and table + cluster.schemaChange("CREATE KEYSPACE test_ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}"); + cluster.schemaChange("CREATE TABLE test_ks.test_tbl (pk int PRIMARY KEY, v int)"); + + // Verify DESCRIBE TABLE does NOT show auto_repair when flag is disabled + // DESCRIBE TABLE result columns: keyspace_name (0), type (1), name (2), create_statement (3) + Object[][] describeResultBefore = cluster.coordinator(1).execute( + "DESCRIBE TABLE test_ks.test_tbl", + ConsistencyLevel.LOCAL_ONE); + String createStatementBefore = (String) describeResultBefore[0][3]; + assertFalse("DESCRIBE TABLE should NOT include auto_repair when flag is disabled", + createStatementBefore.contains("auto_repair")); + + // Verify system_distributed auto-repair tables do NOT exist when flag is disabled + Object[][] historyResultBefore = cluster.coordinator(1).execute( + "SELECT table_name FROM system_schema.tables WHERE keyspace_name = 'system_distributed' AND table_name = 'auto_repair_history'", + ConsistencyLevel.LOCAL_ONE); + assertTrue("auto_repair_history table should NOT exist when flag is disabled", historyResultBefore.length == 0); + + Object[][] priorityResultBefore = cluster.coordinator(1).execute( + "SELECT table_name FROM system_schema.tables WHERE keyspace_name = 'system_distributed' AND table_name = 'auto_repair_priority'", + ConsistencyLevel.LOCAL_ONE); + assertTrue("auto_repair_priority table should NOT exist when flag is disabled", priorityResultBefore.length == 0); + + // Phase 2: Restart with AUTOREPAIR_ENABLE=true + cluster.get(1).nodetoolResult("drain").asserts().success(); + cluster.get(1).shutdown().get(); + CassandraRelevantProperties.AUTOREPAIR_ENABLE.setBoolean(true); + cluster.get(1).startup(); + + // Enable auto-repair scheduling so DESCRIBE TABLE will show the auto_repair property + cluster.get(1).runOnInstance(() -> { + org.apache.cassandra.config.DatabaseDescriptor.getAutoRepairConfig().setAutoRepairSchedulingEnabled(true); + }); + + // Verify DESCRIBE TABLE now shows auto_repair when flag is enabled + Object[][] describeResultAfter = cluster.coordinator(1).execute( + "DESCRIBE TABLE test_ks.test_tbl", + ConsistencyLevel.LOCAL_ONE); + String createStatementAfter = (String) describeResultAfter[0][3]; + assertTrue("DESCRIBE TABLE should include auto_repair after enabling flag", + createStatementAfter.contains("auto_repair")); + + // Verify the test table is still accessible + cluster.coordinator(1).execute("INSERT INTO test_ks.test_tbl (pk, v) VALUES (1, 100)", ConsistencyLevel.ONE); + Object[][] data = cluster.coordinator(1).execute("SELECT * FROM test_ks.test_tbl WHERE pk = 1", ConsistencyLevel.ONE); + assertTrue("Table should still be accessible after enabling flag", data.length == 1); + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerDisallowParallelReplicaRepairAcrossSchedulesTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerDisallowParallelReplicaRepairAcrossSchedulesTest.java new file mode 100644 index 000000000000..e1ccbcb8da1b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerDisallowParallelReplicaRepairAcrossSchedulesTest.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.repair; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.ImmutableMap; + +import org.apache.cassandra.Util; +import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.repair.autorepair.AutoRepair; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.utils.FBUtilities; + +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertEquals; + +/** + * Distributed tests for {@link org.apache.cassandra.repair.autorepair.AutoRepair} scheduler's + * allow_parallel_replica_repair_across_schedules feature. + */ +public class AutoRepairSchedulerDisallowParallelReplicaRepairAcrossSchedulesTest extends TestBaseImpl +{ + private static Cluster cluster; + + @BeforeClass + public static void init() throws IOException + { + // Configure a cluster with preview and incremental repair enabled in a way that preview repair can be + // run on all three nodes concurrently, but incremental repair can only be run when there are no parallel + // repairs. We should detect contention in the incremental repair scheduler but not preview repaired + // scheduler as a result. + cluster = Cluster.build(3) + .withConfig(config -> config + .set("auto_repair", + ImmutableMap.of( + "repair_type_overrides", + ImmutableMap.of(AutoRepairConfig.RepairType.PREVIEW_REPAIRED.getConfigName(), + ImmutableMap.of( + // Configure preview repair to run frequently to + // provoke contention with incremental scheduler. + "initial_scheduler_delay", "5s", + "enabled", "true", + "parallel_repair_count", "3", + "allow_parallel_replica_repair", "true", + "min_repair_interval", "5s"), + AutoRepairConfig.RepairType.INCREMENTAL.getConfigName(), + ImmutableMap.of( + "initial_scheduler_delay", "5s", + "enabled", "true", + "parallel_repair_count", "3", + // Don't allow parallel replica repair across + // schedules + "allow_parallel_replica_repair", "false", + "allow_parallel_replica_repair_across_schedules", "false", + "min_repair_interval", "5s")))) + .set("auto_repair.enabled", "true") + .set("auto_repair.global_settings.repair_retry_backoff", "5s") + .set("auto_repair.repair_task_min_duration", "0s") + .set("auto_repair.repair_check_interval", "5s")) + .start(); + + cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck text, v1 int, v2 int, PRIMARY KEY (pk, ck)) WITH read_repair='NONE'")); + } + + @AfterClass + public static void tearDown() + { + cluster.close(); + } + + @Test + public void testScheduler() + { + cluster.forEach(i -> i.runOnInstance(() -> { + try + { + AutoRepairService.setup(); + AutoRepair.instance.setup(); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + })); + + // validate that the repair ran on all nodes + cluster.forEach(i -> i.runOnInstance(() -> { + // Expect contention on incremental repair across schedules + AutoRepairMetrics incrementalMetrics = AutoRepairMetricsManager.getMetrics(AutoRepairConfig.RepairType.INCREMENTAL); + Util.spinAssert(String.format("%s: AutoRepair has not observed any replica contention in INCREMENTAL repair", + FBUtilities.getJustBroadcastAddress().toString()), + greaterThan(0L), + incrementalMetrics.repairDelayedBySchedule::getCount, + 5, + TimeUnit.MINUTES); + + // No repair contention should be observed for preview repaired since allow_parallel_replica_repair was true + AutoRepairMetrics previewMetrics = AutoRepairMetricsManager.getMetrics(AutoRepairConfig.RepairType.PREVIEW_REPAIRED); + assertEquals(0L, previewMetrics.repairDelayedByReplica.getCount()); + assertEquals(0L, previewMetrics.repairDelayedBySchedule.getCount()); + })); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerStatsHelper.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerStatsHelper.java new file mode 100644 index 000000000000..2bf6aea97a27 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerStatsHelper.java @@ -0,0 +1,260 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.repair; + +import java.io.IOException; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; + +import com.google.common.collect.ImmutableMap; +import org.junit.Assert; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.TokenSupplier; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.repair.autorepair.AutoRepair; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitterTest; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SystemDistributedKeyspace; +import org.apache.cassandra.service.AutoRepairService; + +import static org.apache.cassandra.schema.SchemaConstants.DISTRIBUTED_KEYSPACE_NAME; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Helper class to test {@code totalBytesToRepair}, {@code bytesAlreadyRepaired}, {@code totalKeyspaceRepairPlansToRepair}, + * and {@code keyspaceRepairPlansAlreadyRepaired} + * for {@link org.apache.cassandra.repair.autorepair.AutoRepairState} scheduler + */ +public class AutoRepairSchedulerStatsHelper extends TestBaseImpl +{ + static final Logger logger = LoggerFactory.getLogger(AutoRepairSchedulerStatsHelper.class); + + private static Cluster cluster; + static SimpleDateFormat sdf; + private static final String KEYSPACE1 = "ks1"; + private static final String KEYSPACE2 = "ks2"; + private static final String TABLE1 = "tbl1"; + private static final String TABLE2 = "tbl2"; + + public static void init(int numTokens) throws IOException + { + // Define the expected date format pattern + String pattern = "EEE MMM dd HH:mm:ss z yyyy"; + // Create SimpleDateFormat object with the given pattern + sdf = new SimpleDateFormat(pattern); + sdf.setLenient(false); + CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + cluster = Cluster.build(1) + .withTokenCount(numTokens) + .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(1, numTokens)) + .withConfig(config -> config + .set("num_tokens", numTokens) + .set("auto_repair", + ImmutableMap.of( + "repair_type_overrides", + ImmutableMap.of(AutoRepairConfig.RepairType.FULL.getConfigName(), + ImmutableMap.of( + "initial_scheduler_delay", "5s", + "enabled", "true", + "parallel_repair_count", "1", + // Allow parallel replica repair to allow replicas + // to execute full repair at same time. + "allow_parallel_replica_repair", "true", + // Set min_repair_interval to a higher number to + // run only one round of AutoRepair + "min_repair_interval", "48h")))) + .set("auto_repair.enabled", "true") + .set("auto_repair.global_settings.repair_retry_backoff", "5s") + .set("auto_repair.repair_task_min_duration", "0s") + .set("auto_repair.repair_check_interval", "5s")) + .start(); + + cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE1 + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};"); + cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE2 + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};"); + // disable the compression to calculate an accurate expected repair bytes because with compression enabled, + // we only get estimated bytes, which hinders the ability to do actual vs. expected checks in the test case + cluster.schemaChange(String.format("CREATE TABLE %s.%s (pk int, ck int, v int, PRIMARY KEY (pk, ck)) WITH compression = { 'enabled' : false }", KEYSPACE1, TABLE1)); + cluster.schemaChange(String.format("CREATE TABLE %s.%s (pk int, ck int, v int, PRIMARY KEY (pk, ck)) WITH compression = { 'enabled' : false }", KEYSPACE1, TABLE2)); + cluster.schemaChange(String.format("CREATE TABLE %s.%s (pk int, ck int, v int, PRIMARY KEY (pk, ck)) WITH compression = { 'enabled' : false }", KEYSPACE2, TABLE1)); + cluster.schemaChange(String.format("CREATE TABLE %s.%s (pk int, ck int, v int, PRIMARY KEY (pk, ck)) WITH compression = { 'enabled' : false }", KEYSPACE2, TABLE2)); + } + + public static void tearDown() + { + cluster.close(); + } + + public static void testSchedulerStats() throws ParseException + { + // ensure there was no history of previous repair runs through the scheduler + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY), ConsistencyLevel.QUORUM); + assertEquals(0, rows.length); + + // disabling AutoRepair for system_distributed and system_auth tables to avoid + // interfering with the repaired bytes/plans calculation + disableAutoRepair(SystemDistributedKeyspace.NAME, new HashSet<>(RepairTokenRangeSplitterTest.SYSTEM_DISTRIBUTED_TABLE_NAMES)); + disableAutoRepair(SchemaConstants.AUTH_KEYSPACE_NAME, new HashSet<>(RepairTokenRangeSplitterTest.AUTH_TABLE_NAMES)); + + insertData(); + + cluster.get(1).runOnInstance(() -> { + try + { + AutoRepairService.setup(); + AutoRepair.instance.setup(); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + }); + + cluster.forEach(i -> i.runOnInstance(() -> { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("2s"); + + AutoRepairMetrics fullMetrics = AutoRepairMetricsManager.getMetrics(AutoRepairConfig.RepairType.FULL); + // Since the AutoRepair sleeps up to SLEEP_IF_REPAIR_FINISHES_QUICKLY if the repair finishes quickly, + // so the "nodeRepairTimeInSec" metric should at least be greater than or equal to + // SLEEP_IF_REPAIR_FINISHES_QUICKLY + while (fullMetrics.nodeRepairTimeInSec.getValue().longValue() <= 1) + { + try + { + Thread.sleep(1000); + } + catch (InterruptedException e) + { + throw new RuntimeException(e); + } + } + logger.info("AutoRepair has completed one FULL repair cycle"); + + long expectedRepairBytes = calculateExpectedBytes(Arrays.asList(KEYSPACE1, KEYSPACE2)); + assertEquals(fullMetrics.totalKeyspaceRepairPlansToRepair.getValue(), fullMetrics.keyspaceRepairPlansAlreadyRepaired.getValue()); + // AutoRepair creates a repair plan per keyspace; + // Since there are two separate keyspaces, KEYSPACE1 and KEYSPACE2, the total expected plans should be "2" + assertEquals(2, fullMetrics.totalKeyspaceRepairPlansToRepair.getValue().intValue()); + assertEquals(fullMetrics.totalBytesToRepair.getValue().longValue(), fullMetrics.bytesAlreadyRepaired.getValue().longValue()); + assertEquals(expectedRepairBytes, fullMetrics.bytesAlreadyRepaired.getValue().longValue()); + })); + validate(AutoRepairConfig.RepairType.FULL.toString()); + } + + private static long calculateExpectedBytes(List keyspaces) + { + long totalBytes = 0; + for (String keyspace : keyspaces) + { + for (String table : Arrays.asList(TABLE1, TABLE2)) + { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspace, table); + assertNotNull(cfs); + Iterable sstables = cfs.getTracker().getView().select(SSTableSet.CANONICAL); + List> tokenRanges = AutoRepairUtils.getTokenRanges(true, keyspace); + for (SSTableReader sstable : sstables) + { + long sstableSize = sstable.bytesOnDisk(); + for (Range tokenRange : tokenRanges) + { + // Use SSTableReader API to get positions and size for the range + // This matches what AutoRepairUtils.getSizesForRangeOfSSTables() does + List positionBounds = sstable.getPositionsForRanges(Collections.singleton(tokenRange)); + if (!positionBounds.isEmpty()) + { + long approximateRangeBytesInSSTable = sstable.onDiskSizeForPartitionPositions(positionBounds); + totalBytes += Math.min(approximateRangeBytesInSSTable, sstableSize); + } + } + } + } + } + return totalBytes; + } + + private static void insertData() + { + for (int i = 0; i < 100; i++) + { + cluster.coordinator(1).execute(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (?,?,?)", KEYSPACE1, TABLE1), + ConsistencyLevel.ONE, i, i, i); + cluster.coordinator(1).execute(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (?,?,?)", KEYSPACE1, TABLE2), + ConsistencyLevel.ONE, i, i, i); + cluster.coordinator(1).execute(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (?,?,?)", KEYSPACE2, TABLE1), + ConsistencyLevel.ONE, i, i, i); + cluster.coordinator(1).execute(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (?,?,?)", KEYSPACE2, TABLE2), + ConsistencyLevel.ONE, i, i, i); + } + cluster.get(1).nodetool("flush", KEYSPACE1, TABLE1); + cluster.get(1).nodetool("flush", KEYSPACE1, TABLE2); + cluster.get(1).nodetool("flush", KEYSPACE2, TABLE1); + cluster.get(1).nodetool("flush", KEYSPACE2, TABLE2); + } + + private static void disableAutoRepair(String keyspaceName, Set distributedSystemTables) + { + for (String tableName : distributedSystemTables) + { + cluster.coordinator(1).execute(String.format("ALTER TABLE %s.%s WITH auto_repair = {'full_enabled': 'false'}", keyspaceName, tableName), + ConsistencyLevel.ONE); + } + } + + private static void validate(String repairType) throws ParseException + { + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s where repair_type='%s'", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType), ConsistencyLevel.QUORUM); + assertEquals(1, rows.length); + for (int node = 0; node < rows.length; node++) + { + Object[] row = rows[node]; + // repair_type + Assert.assertEquals(repairType, row[0].toString()); + // host_id + Assert.assertNotNull(UUID.fromString(row[1].toString())); + // ensure there is a legit repair_start_ts and repair_finish_ts + sdf.parse(row[2].toString()); + sdf.parse(row[3].toString()); + // the reason why the repair was scheduled + Assert.assertNotNull(row[4]); + Assert.assertEquals("MY_TURN", row[4].toString()); + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerStatsNoVNodesTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerStatsNoVNodesTest.java new file mode 100644 index 000000000000..96376095b903 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerStatsNoVNodesTest.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.repair; + +import java.io.IOException; +import java.text.ParseException; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.distributed.test.TestBaseImpl; + +/** + * Test {@code totalBytesToRepair}, {@code bytesAlreadyRepaired}, {@code totalKeyspaceRepairPlansToRepair}, + * and {@code keyspaceRepairPlansAlreadyRepaired} + * for {@link org.apache.cassandra.repair.autorepair.AutoRepairState} scheduler without v-nodes + */ +public class AutoRepairSchedulerStatsNoVNodesTest extends TestBaseImpl +{ + @BeforeClass + public static void init() throws IOException + { + AutoRepairSchedulerStatsHelper.init(1); + } + + @AfterClass + public static void tearDown() + { + AutoRepairSchedulerStatsHelper.tearDown(); + } + + @Test + public void testSchedulerStats() throws ParseException + { + AutoRepairSchedulerStatsHelper.testSchedulerStats(); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerStatsVNodesTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerStatsVNodesTest.java new file mode 100644 index 000000000000..8040a3032803 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerStatsVNodesTest.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.repair; + +import java.io.IOException; +import java.text.ParseException; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.distributed.test.TestBaseImpl; + +/** + * Test {@code totalBytesToRepair}, {@code bytesAlreadyRepaired}, {@code totalKeyspaceRepairPlansToRepair}, + * and {@code keyspaceRepairPlansAlreadyRepaired} + * for {@link org.apache.cassandra.repair.autorepair.AutoRepairState} scheduler with v-nodes + */ +public class AutoRepairSchedulerStatsVNodesTest extends TestBaseImpl +{ + @BeforeClass + public static void init() throws IOException + { + AutoRepairSchedulerStatsHelper.init(16); + } + + @AfterClass + public static void tearDown() + { + AutoRepairSchedulerStatsHelper.tearDown(); + } + + @Test + public void testSchedulerStats() throws ParseException + { + AutoRepairSchedulerStatsHelper.testSchedulerStats(); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java new file mode 100644 index 000000000000..adca5070828e --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.repair; + +import java.io.IOException; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.ImmutableMap; + +import org.apache.cassandra.Util; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.distributed.api.TokenSupplier; +import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.schema.SystemDistributedKeyspace; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.repair.autorepair.AutoRepair; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.schema.SchemaConstants.DISTRIBUTED_KEYSPACE_NAME; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.junit.Assert.assertEquals; + +/** + * Distributed tests for {@link org.apache.cassandra.repair.autorepair.AutoRepair} scheduler + */ +public class AutoRepairSchedulerTest extends TestBaseImpl +{ + private static Cluster cluster; + static SimpleDateFormat sdf; + + @BeforeClass + public static void init() throws IOException + { + // Define the expected date format pattern + String pattern = "EEE MMM dd HH:mm:ss z yyyy"; + // Create SimpleDateFormat object with the given pattern + sdf = new SimpleDateFormat(pattern); + sdf.setLenient(false); + // Configure a 3-node cluster with num_tokens: 4 and auto_repair enabled + cluster = Cluster.build(3) + .withTokenCount(4) + .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3, 4)) + .withConfig(config -> config + .set("num_tokens", 4) + .set("auto_repair", + ImmutableMap.of( + "repair_type_overrides", + ImmutableMap.of(AutoRepairConfig.RepairType.FULL.getConfigName(), + ImmutableMap.of( + "initial_scheduler_delay", "5s", + "enabled", "true", + "parallel_repair_count", "3", + // Allow parallel replica repair to allow replicas + // to execute full repair at same time. + "allow_parallel_replica_repair", "true", + "min_repair_interval", "5s"), + AutoRepairConfig.RepairType.INCREMENTAL.getConfigName(), + ImmutableMap.of( + "initial_scheduler_delay", "5s", + "enabled", "true", + // Set parallel repair count to 3 to provoke + // contention between replicas when scheduling. + "parallel_repair_count", "3", + // Disallow parallel replica repair to prevent + // replicas from issuing incremental repair at + // same time. + "allow_parallel_replica_repair", "false", + // Run more aggressively since full repair is + // less restrictive about when it can run repair, + // so need to check more frequently to allow + // incremental to get an attempt in. + "min_repair_interval", "5s")))) + .set("auto_repair.enabled", "true") + .set("auto_repair.global_settings.repair_by_keyspace", "true") + .set("auto_repair.global_settings.repair_retry_backoff", "5s") + .set("auto_repair.repair_task_min_duration", "0s") + .set("auto_repair.repair_check_interval", "5s")) + .start(); + + cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck text, v1 int, v2 int, PRIMARY KEY (pk, ck)) WITH read_repair='NONE'")); + } + + @AfterClass + public static void tearDown() + { + cluster.close(); + } + + @Test + public void testScheduler() throws ParseException + { + // ensure there was no history of previous repair runs through the scheduler + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY), ConsistencyLevel.QUORUM); + assertEquals(0, rows.length); + + cluster.forEach(i -> i.runOnInstance(() -> { + try + { + AutoRepairService.setup(); + AutoRepair.instance.setup(); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + })); + + // validate that the repair ran on all nodes + cluster.forEach(i -> i.runOnInstance(() -> { + String broadcastAddress = FBUtilities.getJustBroadcastAddress().toString(); + + // Reduce sleeping if repair finishes quickly to speed up test but make it non-zero to provoke some + // contention. + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("2s"); + + AutoRepairMetrics incrementalMetrics = AutoRepairMetricsManager.getMetrics(AutoRepairConfig.RepairType.INCREMENTAL); + // Since the AutoRepair sleeps up to SLEEP_IF_REPAIR_FINISHES_QUICKLY if the repair finishes quickly, + // so the "nodeRepairTimeInSec" metric should at least be greater than or equal to + // SLEEP_IF_REPAIR_FINISHES_QUICKLY + Util.spinAssert(String.format("%s: AutoRepair has not yet completed one INCREMENTAL repair cycle", broadcastAddress), + greaterThanOrEqualTo(2L), + () -> incrementalMetrics.nodeRepairTimeInSec.getValue().longValue(), + 5, + TimeUnit.MINUTES); + + // Expect some contention on incremental repair. + Util.spinAssert(String.format("%s: AutoRepair has not observed any replica contention in INCREMENTAL repair", broadcastAddress), + greaterThan(0L), + incrementalMetrics.repairDelayedByReplica::getCount, + 5, + TimeUnit.MINUTES); + // Do not expect any contention across schedules since allow_parallel_replica_repairs across schedules + // was not configured. + assertEquals(0L, incrementalMetrics.repairDelayedBySchedule.getCount()); + + AutoRepairMetrics fullMetrics = AutoRepairMetricsManager.getMetrics(AutoRepairConfig.RepairType.FULL); + Util.spinAssert(String.format("%s: AutoRepair has not yet completed one FULL repair cycle", broadcastAddress), + greaterThanOrEqualTo(2L), + () -> fullMetrics.nodeRepairTimeInSec.getValue().longValue(), + 5, + TimeUnit.MINUTES); + + // No repair contention should be observed for full repair since allow_parallel_replica_repair was true + assertEquals(0L, fullMetrics.repairDelayedByReplica.getCount()); + assertEquals(0L, fullMetrics.repairDelayedBySchedule.getCount()); + })); + + validate(AutoRepairConfig.RepairType.FULL.toString()); + validate(AutoRepairConfig.RepairType.INCREMENTAL.toString()); + } + + private void validate(String repairType) throws ParseException + { + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s where repair_type='%s'", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType), ConsistencyLevel.QUORUM); + assertEquals(3, rows.length); + for (int node = 0; node < rows.length; node++) + { + Object[] row = rows[node]; + // repair_type + Assert.assertEquals(repairType, row[0].toString()); + // host_id + Assert.assertNotNull(UUID.fromString(row[1].toString())); + // ensure there is a legit repair_start_ts and repair_finish_ts + sdf.parse(row[2].toString()); + sdf.parse(row[3].toString()); + // the reason why the repair was scheduled + Assert.assertNotNull(row[4]); + Assert.assertEquals("MY_TURN", row[4].toString()); + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairTablePropertyDTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairTablePropertyDTest.java new file mode 100644 index 000000000000..e3c0984749a8 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairTablePropertyDTest.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.repair; + +import java.io.IOException; + +import com.google.common.collect.ImmutableMap; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Distributed tests that verify "auto_repair" column behavior in system_schema.tables. + *

    + * This test verifies that: + * 1. When auto-repair scheduling is enabled, the auto_repair column contains data + * 2. When auto-repair scheduling is disabled, the auto_repair column does not contain data + *

    + * This test uses a distributed cluster to ensure JVM properties are properly initialized + * before the schema is loaded. + */ +public class AutoRepairTablePropertyDTest extends TestBaseImpl +{ + private static Cluster cluster; + + @BeforeClass + public static void init() throws IOException + { + // Ensure AUTOREPAIR_ENABLE is true so the auto_repair column exists in the schema + CassandraRelevantProperties.AUTOREPAIR_ENABLE.setBoolean(true); + + // Configure a single-node cluster with auto_repair enabled + cluster = Cluster.build(1) + .withConfig(config -> config + .set("auto_repair", + ImmutableMap.of( + "repair_type_overrides", + ImmutableMap.of(AutoRepairConfig.RepairType.FULL.getConfigName(), + ImmutableMap.of( + "enabled", "true", + "initial_scheduler_delay", "1h"))))) + .start(); + } + + @AfterClass + public static void teardown() + { + if (cluster != null) + cluster.close(); + } + + @Test + public void testAutoRepairColumnExistsWhenEnabled() + { + // Verify the auto_repair column exists in system_schema.tables + cluster.get(1).runOnInstance(() -> { + org.apache.cassandra.db.ColumnFamilyStore tables = + org.apache.cassandra.db.Keyspace.open(org.apache.cassandra.schema.SchemaConstants.SCHEMA_KEYSPACE_NAME) + .getColumnFamilyStore(org.apache.cassandra.schema.SchemaKeyspaceTables.TABLES); + + org.apache.cassandra.schema.ColumnMetadata autoRepairColumn = + tables.metadata().getColumn(org.apache.cassandra.utils.ByteBufferUtil.bytes("auto_repair")); + + // When AUTOREPAIR_ENABLE is true, the column should exist + assertTrue("auto_repair column should exist when AUTOREPAIR_ENABLE is true", + autoRepairColumn != null); + }); + } + + @Test + public void testAutoRepairColumnDataWrittenWhenSchedulerEnabled() + { + // When auto-repair scheduling is enabled, verify data is written to the auto_repair column + cluster.get(1).runOnInstance(() -> { + // Enable scheduling + org.apache.cassandra.config.DatabaseDescriptor.getAutoRepairConfig().setAutoRepairSchedulingEnabled(true); + + org.apache.cassandra.db.ColumnFamilyStore tables = + org.apache.cassandra.db.Keyspace.open(org.apache.cassandra.schema.SchemaConstants.SCHEMA_KEYSPACE_NAME) + .getColumnFamilyStore(org.apache.cassandra.schema.SchemaKeyspaceTables.TABLES); + + org.apache.cassandra.schema.ColumnMetadata autoRepairColumn = + tables.metadata().getColumn(org.apache.cassandra.utils.ByteBufferUtil.bytes("auto_repair")); + + org.apache.cassandra.db.SimpleBuilders.RowBuilder builder = + new org.apache.cassandra.db.SimpleBuilders.RowBuilder(tables.metadata(), "table_name"); + org.apache.cassandra.schema.SchemaKeyspace.addTableParamsToRowBuilder(tables.metadata().params, builder); + org.apache.cassandra.db.rows.Row row = builder.build(); + + org.apache.cassandra.db.rows.ColumnData data = row.getCell(autoRepairColumn); + + // When scheduling is enabled, data should be written + assertTrue("auto_repair data should be written when scheduling is enabled", data != null); + }); + } + + @Test + public void testAutoRepairColumnDataNotWrittenWhenSchedulerDisabled() + { + // When auto-repair scheduling is disabled, verify no data is written to the auto_repair column + cluster.get(1).runOnInstance(() -> { + // Disable scheduling + org.apache.cassandra.config.DatabaseDescriptor.getAutoRepairConfig().setAutoRepairSchedulingEnabled(false); + + org.apache.cassandra.db.ColumnFamilyStore tables = + org.apache.cassandra.db.Keyspace.open(org.apache.cassandra.schema.SchemaConstants.SCHEMA_KEYSPACE_NAME) + .getColumnFamilyStore(org.apache.cassandra.schema.SchemaKeyspaceTables.TABLES); + + org.apache.cassandra.schema.ColumnMetadata autoRepairColumn = + tables.metadata().getColumn(org.apache.cassandra.utils.ByteBufferUtil.bytes("auto_repair")); + + org.apache.cassandra.db.SimpleBuilders.RowBuilder builder = + new org.apache.cassandra.db.SimpleBuilders.RowBuilder(tables.metadata(), "table_name"); + org.apache.cassandra.schema.SchemaKeyspace.addTableParamsToRowBuilder(tables.metadata().params, builder); + org.apache.cassandra.db.rows.Row row = builder.build(); + + org.apache.cassandra.db.rows.ColumnData data = row.getCell(autoRepairColumn); + + // When scheduling is disabled, data should NOT be written + assertFalse("auto_repair data should NOT be written when scheduling is disabled", data != null); + }); + } + + @Test + public void testAutoRepairColumnInViewsSchema() + { + // Verify the auto_repair column behavior in system_schema.views as well + cluster.get(1).runOnInstance(() -> { + org.apache.cassandra.db.ColumnFamilyStore views = + org.apache.cassandra.db.Keyspace.open(org.apache.cassandra.schema.SchemaConstants.SCHEMA_KEYSPACE_NAME) + .getColumnFamilyStore(org.apache.cassandra.schema.SchemaKeyspaceTables.VIEWS); + + org.apache.cassandra.schema.ColumnMetadata autoRepairColumn = + views.metadata().getColumn(org.apache.cassandra.utils.ByteBufferUtil.bytes("auto_repair")); + + // When AUTOREPAIR_ENABLE is true, the column should exist in views table too + assertTrue("auto_repair column should exist in views schema when AUTOREPAIR_ENABLE is true", + autoRepairColumn != null); + }); + } +} diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java index 49af3e1271ac..d2526adaf6af 100644 --- a/test/unit/org/apache/cassandra/Util.java +++ b/test/unit/org/apache/cassandra/Util.java @@ -25,6 +25,7 @@ import java.io.IOError; import java.io.IOException; import java.io.InputStream; +import java.lang.reflect.Field; import java.math.BigInteger; import java.net.UnknownHostException; import java.nio.ByteBuffer; @@ -54,6 +55,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.commons.lang3.StringUtils; import org.junit.Assume; import org.slf4j.Logger; @@ -1275,4 +1278,14 @@ public static RuntimeException testMustBeImplementedForSSTableFormat() { return new UnsupportedOperationException("Test must be implemented for sstable format " + DatabaseDescriptor.getSelectedSSTableFormat().getClass().getName()); } + + // Replaces the global auto-repair config with a new config where auto-repair scheduling is enabled/disabled + public static void setAutoRepairEnabled(boolean enabled) throws Exception + { + Config config = DatabaseDescriptor.getRawConfig(); + config.auto_repair = new AutoRepairConfig(enabled); + Field configField = DatabaseDescriptor.class.getDeclaredField("conf"); + configField.setAccessible(true); + configField.set(null, config); + } } diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java index 59a4f5c48c96..ce8a2b026fa8 100644 --- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java +++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java @@ -102,6 +102,16 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.config.ConfigurationLoader", "org.apache.cassandra.config.Config$CorruptedTombstoneStrategy", "org.apache.cassandra.config.Config$BatchlogEndpointStrategy", + "org.apache.cassandra.repair.autorepair.AutoRepairConfig", + "org.apache.cassandra.repair.autorepair.AutoRepairConfig$Options", + "org.apache.cassandra.repair.autorepair.AutoRepairConfig$RepairType", + "org.apache.cassandra.repair.autorepair.AutoRepairState", + "org.apache.cassandra.repair.autorepair.FixedSplitTokenRangeSplitter", + "org.apache.cassandra.repair.autorepair.FullRepairState", + "org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter", + "org.apache.cassandra.repair.autorepair.IncrementalRepairState", + "org.apache.cassandra.repair.autorepair.PreviewRepairedState", + "org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter", "org.apache.cassandra.config.DatabaseDescriptor$ByteUnit", "org.apache.cassandra.config.DataRateSpec", "org.apache.cassandra.config.DataRateSpec$DataRateUnit", @@ -285,6 +295,7 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.utils.concurrent.RefCounted", "org.apache.cassandra.utils.concurrent.SelfRefCounted", "org.apache.cassandra.utils.concurrent.Transactional", + "org.apache.cassandra.utils.progress.ProgressListener", "org.apache.cassandra.utils.concurrent.UncheckedInterruptedException", "org.apache.cassandra.utils.StorageCompatibilityMode" }; diff --git a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java index 11833783a3ad..11dd6e26089c 100644 --- a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java +++ b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java @@ -35,6 +35,7 @@ import org.apache.cassandra.distributed.shared.WithProperties; import org.apache.cassandra.io.util.File; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.yaml.snakeyaml.error.YAMLException; import static org.apache.cassandra.config.CassandraRelevantProperties.CONFIG_ALLOW_SYSTEM_PROPERTIES; @@ -42,6 +43,7 @@ import static org.apache.cassandra.config.YamlConfigurationLoader.SYSTEM_PROPERTY_PREFIX; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -104,8 +106,8 @@ public void validateTypes() assertEquals("You have wrongly defined a config parameter of abstract type DurationSpec, DataStorageSpec or DataRateSpec." + "Please check the config docs, otherwise Cassandra won't be able to start with this parameter being set in cassandra.yaml.", Arrays.stream(Config.class.getFields()) - .filter(f -> !Modifier.isStatic(f.getModifiers())) - .filter(isDurationSpec.or(isDataRateSpec).or(isDataStorageSpec)).count(), 0); + .filter(f -> !Modifier.isStatic(f.getModifiers())) + .filter(isDurationSpec.or(isDataRateSpec).or(isDataStorageSpec)).count(), 0); } @Test @@ -113,12 +115,12 @@ public void updateInPlace() { Config config = new Config(); Map map = ImmutableMap.builder().put("storage_port", 123) - .put("commitlog_sync", Config.CommitLogSync.batch) - .put("seed_provider.class_name", "org.apache.cassandra.locator.SimpleSeedProvider") - .put("client_encryption_options.cipher_suites", Collections.singletonList("FakeCipher")) - .put("client_encryption_options.optional", false) - .put("client_encryption_options.enabled", true) - .build(); + .put("commitlog_sync", Config.CommitLogSync.batch) + .put("seed_provider.class_name", "org.apache.cassandra.locator.SimpleSeedProvider") + .put("client_encryption_options.cipher_suites", Collections.singletonList("FakeCipher")) + .put("client_encryption_options.optional", false) + .put("client_encryption_options.enabled", true) + .build(); Config updated = YamlConfigurationLoader.updateFromMap(map, true, config); assert updated == config : "Config pointers do not match"; assertThat(config.storage_port).isEqualTo(123); @@ -271,6 +273,12 @@ public void fromMapTest() Map encryptionOptions = ImmutableMap.of("cipher_suites", Collections.singletonList("FakeCipher"), "optional", false, "enabled", true); + Map autoRepairConfig = ImmutableMap.of("enabled", true, + "global_settings", + ImmutableMap.of("number_of_repair_threads", 1), + "repair_type_overrides", + ImmutableMap.of("full", + ImmutableMap.of("number_of_repair_threads", 2))); Map map = new ImmutableMap.Builder() .put("storage_port", storagePort) .put("commitlog_sync", commitLogSync) @@ -279,6 +287,7 @@ public void fromMapTest() .put("internode_socket_send_buffer_size", "5B") .put("internode_socket_receive_buffer_size", "5B") .put("commitlog_sync_group_window_in_ms", "42") + .put("auto_repair", autoRepairConfig) .build(); Config config = YamlConfigurationLoader.fromMap(map, Config.class); @@ -289,6 +298,9 @@ public void fromMapTest() assertEquals(true, config.client_encryption_options.enabled); // Check a nested object assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_send_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_receive_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) + assertTrue(config.auto_repair.enabled); + assertEquals(new DurationSpec.IntSecondsBound("6h"), config.auto_repair.getAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType.INCREMENTAL)); + config.auto_repair.setMaterializedViewRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, false); } @Test @@ -485,4 +497,4 @@ public static Config load(String path) } return new YamlConfigurationLoader().loadConfig(url); } -} +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/cql3/SystemKeyspaceTablesNamesTest.java b/test/unit/org/apache/cassandra/cql3/SystemKeyspaceTablesNamesTest.java index fa7404c87ccd..1e25960dff55 100644 --- a/test/unit/org/apache/cassandra/cql3/SystemKeyspaceTablesNamesTest.java +++ b/test/unit/org/apache/cassandra/cql3/SystemKeyspaceTablesNamesTest.java @@ -84,8 +84,8 @@ public void testSystemAuthKeyspaceTableNames() public void testSystemDistributedKeyspaceTableNames() { assertExpectedTablesInKeyspace(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - "SystemDistributedKeyspace.TABLE_NAMES", - SystemDistributedKeyspace.TABLE_NAMES); + "SystemDistributedKeyspace.getTableNames()", + SystemDistributedKeyspace.getTableNames()); } private static void assertExpectedTablesInKeyspace(String keyspaceName, String expectedTableSource, Set expectedTables) diff --git a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java index 572df6c03be3..17b32e30fe57 100644 --- a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java @@ -267,6 +267,19 @@ public void testDescribeVirtualTables() throws Throwable @Test public void testDescribe() throws Throwable + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairSchedulingEnabled(false); + helperTestDescribe(); + } + + @Test + public void testDescribeWithAutoRepair() throws Throwable + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairSchedulingEnabled(true); + helperTestDescribe(); + } + + public void helperTestDescribe() throws Throwable { try { @@ -1055,25 +1068,51 @@ private static String testTableOutput() private static String tableParametersCql() { - return "additional_write_policy = '99p'\n" + - " AND allow_auto_snapshot = true\n" + - " AND bloom_filter_fp_chance = 0.01\n" + - " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" + - " AND cdc = false\n" + - " AND comment = ''\n" + - " AND compaction = " + cqlQuoted(CompactionParams.DEFAULT.asMap()) + "\n" + - " AND compression = {'chunk_length_in_kb': '16', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n" + - " AND memtable = 'default'\n" + - " AND crc_check_chance = 1.0\n" + - " AND default_time_to_live = 0\n" + - " AND extensions = {}\n" + - " AND gc_grace_seconds = 864000\n" + - " AND incremental_backups = true\n" + - " AND max_index_interval = 2048\n" + - " AND memtable_flush_period_in_ms = 0\n" + - " AND min_index_interval = 128\n" + - " AND read_repair = 'BLOCKING'\n" + - " AND speculative_retry = '99p';"; + if (!DatabaseDescriptor.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) + { + return "additional_write_policy = '99p'\n" + + " AND allow_auto_snapshot = true\n" + + " AND bloom_filter_fp_chance = 0.01\n" + + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" + + " AND cdc = false\n" + + " AND comment = ''\n" + + " AND compaction = " + cqlQuoted(CompactionParams.DEFAULT.asMap()) + "\n" + + " AND compression = {'chunk_length_in_kb': '16', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n" + + " AND memtable = 'default'\n" + + " AND crc_check_chance = 1.0\n" + + " AND default_time_to_live = 0\n" + + " AND extensions = {}\n" + + " AND gc_grace_seconds = 864000\n" + + " AND incremental_backups = true\n" + + " AND max_index_interval = 2048\n" + + " AND memtable_flush_period_in_ms = 0\n" + + " AND min_index_interval = 128\n" + + " AND read_repair = 'BLOCKING'\n" + + " AND speculative_retry = '99p';"; + } + else + { + return "additional_write_policy = '99p'\n" + + " AND allow_auto_snapshot = true\n" + + " AND bloom_filter_fp_chance = 0.01\n" + + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" + + " AND cdc = false\n" + + " AND comment = ''\n" + + " AND compaction = " + cqlQuoted(CompactionParams.DEFAULT.asMap()) + "\n" + + " AND compression = {'chunk_length_in_kb': '16', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n" + + " AND memtable = 'default'\n" + + " AND crc_check_chance = 1.0\n" + + " AND default_time_to_live = 0\n" + + " AND extensions = {}\n" + + " AND gc_grace_seconds = 864000\n" + + " AND incremental_backups = true\n" + + " AND max_index_interval = 2048\n" + + " AND memtable_flush_period_in_ms = 0\n" + + " AND min_index_interval = 128\n" + + " AND read_repair = 'BLOCKING'\n" + + " AND speculative_retry = '99p'\n" + + " AND auto_repair = {'full_enabled': 'true', 'incremental_enabled': 'true', 'preview_repaired_enabled': 'true', 'priority': '0'};"; + } } private static String cqlQuoted(Map map) @@ -1083,24 +1122,49 @@ private static String cqlQuoted(Map map) private static String mvParametersCql() { - return "additional_write_policy = '99p'\n" + - " AND allow_auto_snapshot = true\n" + - " AND bloom_filter_fp_chance = 0.01\n" + - " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" + - " AND cdc = false\n" + - " AND comment = ''\n" + - " AND compaction = " + cqlQuoted(CompactionParams.DEFAULT.asMap()) + "\n" + - " AND compression = {'chunk_length_in_kb': '16', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n" + - " AND memtable = 'default'\n" + - " AND crc_check_chance = 1.0\n" + - " AND extensions = {}\n" + - " AND gc_grace_seconds = 864000\n" + - " AND incremental_backups = true\n" + - " AND max_index_interval = 2048\n" + - " AND memtable_flush_period_in_ms = 0\n" + - " AND min_index_interval = 128\n" + - " AND read_repair = 'BLOCKING'\n" + - " AND speculative_retry = '99p';"; + if (!DatabaseDescriptor.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) + { + return "additional_write_policy = '99p'\n" + + " AND allow_auto_snapshot = true\n" + + " AND bloom_filter_fp_chance = 0.01\n" + + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" + + " AND cdc = false\n" + + " AND comment = ''\n" + + " AND compaction = " + cqlQuoted(CompactionParams.DEFAULT.asMap()) + "\n" + + " AND compression = {'chunk_length_in_kb': '16', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n" + + " AND memtable = 'default'\n" + + " AND crc_check_chance = 1.0\n" + + " AND extensions = {}\n" + + " AND gc_grace_seconds = 864000\n" + + " AND incremental_backups = true\n" + + " AND max_index_interval = 2048\n" + + " AND memtable_flush_period_in_ms = 0\n" + + " AND min_index_interval = 128\n" + + " AND read_repair = 'BLOCKING'\n" + + " AND speculative_retry = '99p';"; + } + else + { + return "additional_write_policy = '99p'\n" + + " AND allow_auto_snapshot = true\n" + + " AND bloom_filter_fp_chance = 0.01\n" + + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" + + " AND cdc = false\n" + + " AND comment = ''\n" + + " AND compaction = " + cqlQuoted(CompactionParams.DEFAULT.asMap()) + "\n" + + " AND compression = {'chunk_length_in_kb': '16', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n" + + " AND memtable = 'default'\n" + + " AND crc_check_chance = 1.0\n" + + " AND extensions = {}\n" + + " AND gc_grace_seconds = 864000\n" + + " AND incremental_backups = true\n" + + " AND max_index_interval = 2048\n" + + " AND memtable_flush_period_in_ms = 0\n" + + " AND min_index_interval = 128\n" + + " AND read_repair = 'BLOCKING'\n" + + " AND speculative_retry = '99p'\n" + + " AND auto_repair = {'full_enabled': 'true', 'incremental_enabled': 'true', 'preview_repaired_enabled': 'true', 'priority': '0'};"; + } } private static String keyspaceOutput() diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamReceiverTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamReceiverTest.java new file mode 100644 index 000000000000..466cfe9eab21 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamReceiverTest.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.streaming; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.streaming.StreamOperation; +import org.apache.cassandra.streaming.StreamSession; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link org.apache.cassandra.db.streaming.CassandraStreamReceiver} + */ +public class CassandraStreamReceiverTest extends CQLTester +{ + @Mock + private StreamSession session; + + private static final String CDC_TABLE = "cdc_table"; + private static final String MV_TABLE = "mv_table"; + private static final String CDC_MV_TABLE = "cdc_mv_table"; + private static final String NO_CDC_MV_TABLE = "no_cdc_mv_table"; + + @Before + public void setup() + { + // Set cdc_on_repair_enabled materialized_views_on_repair to true + DatabaseDescriptor.setCDCOnRepairEnabled(true); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); + // Enable materialized views + DatabaseDescriptor.setMaterializedViewsEnabled(true); + + MockitoAnnotations.initMocks(this); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int) WITH cdc=true", KEYSPACE, CDC_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int) WITH cdc=false", KEYSPACE, MV_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW IF NOT EXISTS %s.mv AS SELECT * FROM %s.%s WHERE pk IS NOT NULL PRIMARY KEY (pk)", KEYSPACE, KEYSPACE, MV_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int) WITH cdc=true", KEYSPACE, CDC_MV_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW IF NOT EXISTS %s.mv2 AS SELECT * FROM %s.%s WHERE pk IS NOT NULL PRIMARY KEY (pk)", KEYSPACE, KEYSPACE, CDC_MV_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int) WITH cdc=false", KEYSPACE, NO_CDC_MV_TABLE)); + } + + @Test + public void testRequiresWritePathRepair() + { + // given a CDC table with a materialized view attached to it. + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CDC_MV_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.REPAIR); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + // Should require write path since cdc_on_repair_enabled and materialized_views_on_repair_enabled are both true. + assertTrue(receiver.requiresWritePath(cfs)); + } + + @Test + public void testRequiresWritePathBulkLoad() + { + // given a CDC table with a materialized view attached to it. + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CDC_MV_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.BULK_LOAD); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + // Should require write path since cdc_on_repair_enabled and materialized_views_on_repair_enabled are both true. + assertTrue(receiver.requiresWritePath(cfs)); + } + + @Test + public void testDoesNotRequireWritePathNoCDCOrMV() + { + // Given cdc_on_repaired_enabled and materialized_views_on_repair_enabled are false + // requiresWritePath should still return false for a non-CDC table. + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(NO_CDC_MV_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.BULK_LOAD); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + assertFalse(receiver.requiresWritePath(cfs)); + } + + @Test + public void testRequiresWritePathRepairMVOnly() + { + // Given cdc_on_repaired_enabled and materialized_views_on_repair_enabled are true + // requiresWritePath should return true for a table with materialized views. + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(MV_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.REPAIR); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + assertTrue(receiver.requiresWritePath(cfs)); + } + + @Test + public void testRequiresWritePathRepairCDCOnRepairEnabled() + { + // Given cdc_on_repaired_enabled and materialized_views_on_repair_enabled are true + // requiresWritePath should return true for a table with CDC enabled. + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CDC_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.REPAIR); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + assertTrue(receiver.requiresWritePath(cfs)); + + System.clearProperty("cassandra.streaming.requires_cdc_replay"); + } + + @Test + public void testDoesNotRequireWritePathRepairCDCOnRepairEnabledFalse() + { + // Given cdc_on_repaired_enabled and materialized_views_on_repair_enabled are false + // requiresWritePath should return false for a table with CDC enabled. + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CDC_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.BULK_LOAD); + CassandraStreamReceiver receiver1 = new CassandraStreamReceiver(cfs, session, 1); + assertFalse(receiver1.requiresWritePath(cfs)); + + // When flipping cdc_on_repair_enabled to true + // requiresWritePath should return true. + DatabaseDescriptor.setCDCOnRepairEnabled(true); + CassandraStreamReceiver receiver2 = new CassandraStreamReceiver(cfs, session, 1); + assertTrue(receiver2.requiresWritePath(cfs)); + } +} diff --git a/test/unit/org/apache/cassandra/repair/RepairCoordinatorTest.java b/test/unit/org/apache/cassandra/repair/RepairCoordinatorTest.java new file mode 100644 index 000000000000..6eda22c446b0 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/RepairCoordinatorTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.repair.messages.RepairOption; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.streaming.PreviewKind; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.mockito.Mockito.when; + + +public class RepairCoordinatorTest extends CQLTester +{ + @Mock + StorageService storageService; + + @Mock + RepairOption repairOption; + + @Mock + Throwable throwable; + + @Before + public void setUp() + { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testFailNPE() + { + RepairCoordinator repairCoordinator = new RepairCoordinator(storageService, 0, repairOption, KEYSPACE); + repairCoordinator.firstError.set(throwable); + when(throwable.getMessage()).thenReturn(null); + when(repairOption.getPreviewKind()).thenReturn(PreviewKind.NONE); + + repairCoordinator.fail(null); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigRepairTypeTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigRepairTypeTest.java new file mode 100644 index 000000000000..ec36bb5cfd0c --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigRepairTypeTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Unit tests for {@link org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType} + */ +public class AutoRepairConfigRepairTypeTest +{ + @Test + public void testRepairTypeParsing() + { + Assert.assertEquals(AutoRepairConfig.RepairType.FULL, AutoRepairConfig.RepairType.parse("FULL")); + Assert.assertEquals(AutoRepairConfig.RepairType.FULL, AutoRepairConfig.RepairType.parse("FuLl")); + Assert.assertEquals(AutoRepairConfig.RepairType.FULL, AutoRepairConfig.RepairType.parse("full")); + Assert.assertEquals(AutoRepairConfig.RepairType.INCREMENTAL, AutoRepairConfig.RepairType.parse("INCREMENTAL")); + Assert.assertEquals(AutoRepairConfig.RepairType.INCREMENTAL, AutoRepairConfig.RepairType.parse("incremental")); + Assert.assertEquals(AutoRepairConfig.RepairType.INCREMENTAL, AutoRepairConfig.RepairType.parse("inCRemenTal")); + Assert.assertEquals(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, AutoRepairConfig.RepairType.parse("PREVIEW_REPAIRED")); + Assert.assertEquals(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, AutoRepairConfig.RepairType.parse("preview_repaired")); + Assert.assertEquals(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, AutoRepairConfig.RepairType.parse("Preview_Repaired")); + } + + @Test(expected = NullPointerException.class) + public void testNullRepairTypeParsing() + { + AutoRepairConfig.RepairType.parse(null); + } + + @Test(expected = IllegalArgumentException.class) + public void testEmptyRepairTypeParsing() + { + AutoRepairConfig.RepairType.parse(""); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidRepairTypeParsing() + { + AutoRepairConfig.RepairType.parse("very_FULL"); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java new file mode 100644 index 000000000000..93ef906a287d --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -0,0 +1,509 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.HashSet; +import java.util.Map; +import java.util.Collections; +import java.util.Set; + +import com.google.common.collect.ImmutableSet; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.config.ParameterizedClass; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.Options; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Unit tests for {@link org.apache.cassandra.repair.autorepair.AutoRepairConfig} + */ +@RunWith(Parameterized.class) +public class AutoRepairConfigTest extends CQLTester +{ + private AutoRepairConfig config; + + private final Set testSet = ImmutableSet.of("dc1"); + + @Parameterized.Parameter + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameters + public static Object[] repairTypes() + { + return AutoRepairConfig.RepairType.values(); + } + + @Before + public void setUp() + { + config = new AutoRepairConfig(true); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + } + + @Test + public void autoRepairConfigDefaultsAreNotNull() + { + AutoRepairConfig config = new AutoRepairConfig(); + assertNotNull(config.global_settings); + } + + @Test + public void autoRepairConfigRepairTypesAreNotNull() + { + AutoRepairConfig config = new AutoRepairConfig(); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + assertNotNull(config.getOptions(repairType)); + } + } + + @Test + public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsEnabled() + { + config.global_settings.enabled = true; + + assertTrue(config.isAutoRepairEnabled(repairType)); + } + + @Test + public void testRepairMinDuration() + { + config = new AutoRepairConfig(false); + + config.setRepairTaskMinDuration("3s"); + assertEquals(3L, config.getRepairTaskMinDuration().toSeconds()); + } + + @Test + public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledGlobally() + { + config = new AutoRepairConfig(false); + config.global_settings.enabled = true; + assertFalse(config.isAutoRepairEnabled(repairType)); + } + + @Test + public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledForRepairType() + { + config.global_settings.enabled = true; + config.setAutoRepairEnabled(repairType, false); + assertFalse(config.getOptions(repairType).enabled); + } + + @Test + public void testSetAutoRepairEnabledNoMVOrCDC() + { + DatabaseDescriptor.setCDCEnabled(false); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + config.setAutoRepairEnabled(repairType, true); + + assertTrue(config.getOptions(repairType).enabled); + } + + @Test + public void testSetRepairByKeyspace() + { + // Should default to true. + assertTrue(config.getRepairByKeyspace(repairType)); + + config.setRepairByKeyspace(repairType, false); + + assertFalse(config.getOptions(repairType).repair_by_keyspace); + } + + @Test + public void testGetRepairByKeyspace() + { + config.global_settings.repair_by_keyspace = true; + + boolean result = config.getRepairByKeyspace(repairType); + + assertTrue(result); + } + + @Test + public void testSetRepairThreads() + { + config.setRepairThreads(repairType, 5); + + assertEquals(5, config.getOptions(repairType).number_of_repair_threads.intValue()); + } + + @Test + public void testGetRepairThreads() + { + config.global_settings.number_of_repair_threads = 5; + + int result = config.getRepairThreads(repairType); + + assertEquals(5, result); + } + + @Test + public void testGetRepairMinFrequencyInHours() + { + config.global_settings.min_repair_interval = new DurationSpec.IntSecondsBound("5s"); + + DurationSpec.IntSecondsBound result = config.getRepairMinInterval(repairType); + + assertEquals(5, result.toSeconds()); + } + + @Test + public void testSetRepairMinFrequencyInHours() + { + config.setRepairMinInterval(repairType, "5s"); + + assertEquals(5, config.getOptions(repairType).min_repair_interval.toSeconds()); + } + + @Test + public void testGetAutoRepairHistoryClearDeleteHostsBufferInSec() + { + config.history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound("5s"); + + int result = config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds(); + + assertEquals(5, result); + } + + @Test + public void testSetAutoRepairHistoryClearDeleteHostsBufferInSec() + { + config.setAutoRepairHistoryClearDeleteHostsBufferInterval("5s"); + + assertEquals(new DurationSpec.IntSecondsBound("5s"), config.history_clear_delete_hosts_buffer_interval); + } + + @Test + public void testGetRepairSSTableCountHigherThreshold() + { + config.global_settings.sstable_upper_threshold = 5; + + int result = config.getRepairSSTableCountHigherThreshold(repairType); + + assertEquals(5, result); + } + + @Test + public void testSetRepairSSTableCountHigherThreshold() + { + config.setRepairSSTableCountHigherThreshold(repairType, 5); + + assertEquals(5, config.getOptions(repairType).sstable_upper_threshold.intValue()); + } + + @Test + public void testGetAutoRepairTableMaxRepairTimeInSec() + { + config.global_settings.table_max_repair_time = new DurationSpec.IntSecondsBound("5s"); + + DurationSpec.IntSecondsBound result = config.getAutoRepairTableMaxRepairTime(repairType); + + assertEquals(5, result.toSeconds()); + } + + @Test + public void testSetAutoRepairTableMaxRepairTimeInSec() + { + config.setAutoRepairTableMaxRepairTime(repairType, "5s"); + + assertEquals(5, config.getOptions(repairType).table_max_repair_time.toSeconds()); +} + + @Test + public void testGetIgnoreDCs() + { + config.global_settings.ignore_dcs = testSet; + + Set result = config.getIgnoreDCs(repairType); + + assertEquals(testSet, result); + } + + @Test + public void testSetIgnoreDCs() + { + config.setIgnoreDCs(repairType, testSet); + + assertEquals(config.getOptions(repairType).ignore_dcs, testSet); + } + + @Test + public void testGetRepairPrimaryTokenRangeOnly() + { + config.global_settings.repair_primary_token_range_only = true; + + boolean result = config.getRepairPrimaryTokenRangeOnly(repairType); + + assertTrue(result); + } + + @Test + public void testSetRepairPrimaryTokenRangeOnly() + { + config.setRepairPrimaryTokenRangeOnly(repairType, true); + + assertTrue(config.getOptions(repairType).repair_primary_token_range_only); + } + + @Test + public void testGetParallelRepairPercentageInGroup() + { + config.global_settings.parallel_repair_percentage = 5; + + int result = config.getParallelRepairPercentage(repairType); + + assertEquals(5, result); + } + + @Test + public void testSetParallelRepairPercentageInGroup() + { + config.setParallelRepairPercentage(repairType, 5); + + assertEquals(5, config.getOptions(repairType).parallel_repair_percentage.intValue()); + } + + @Test + public void testGetParallelRepairCountInGroup() + { + config.global_settings.parallel_repair_count = 5; + + int result = config.getParallelRepairCount(repairType); + + assertEquals(5, result); + } + + @Test + public void testSetParallelRepairCountInGroup() + { + config.setParallelRepairCount(repairType, 5); + + assertEquals(5, config.getOptions(repairType).parallel_repair_count.intValue()); + } + + @Test + public void testGetAllowParallelReplicaRepair() + { + // should default to false + assertFalse(config.global_settings.allow_parallel_replica_repair); + assertFalse(config.getAllowParallelReplicaRepair(repairType)); + + // setting global to true should also cause repair type config to inherit. + config.global_settings.allow_parallel_replica_repair = true; + assertTrue(config.getAllowParallelReplicaRepair(repairType)); + + } + + @Test + public void testSetAllowParallelReplicaRepair() + { + // should default to false + assertFalse(config.getAllowParallelReplicaRepair(repairType)); + + // setting explicitly for repair type should update it + config.setAllowParallelReplicaRepair(repairType, true); + assertTrue(config.getAllowParallelReplicaRepair(repairType)); + } + + @Test + public void testGetAllowParallelReplicaRepairAcrossSchedules() + { + // should default to true + assertTrue(config.global_settings.allow_parallel_replica_repair_across_schedules); + assertTrue(config.getAllowParallelReplicaRepairAcrossSchedules(repairType)); + + // setting global to true should also cause repair type config to inherit. + config.global_settings.allow_parallel_replica_repair_across_schedules = false; + assertFalse(config.getAllowParallelReplicaRepairAcrossSchedules(repairType)); + + } + + @Test + public void testSetAllowParallelReplicaRepairAcrossSchedules() + { + // should default to true + assertTrue(config.getAllowParallelReplicaRepairAcrossSchedules(repairType)); + + // setting explicitly for repair type should update it + config.setAllowParallelReplicaRepairAcrossSchedules(repairType, false); + assertFalse(config.getAllowParallelReplicaRepairAcrossSchedules(repairType)); + } + + @Test + public void testGetMaterializedViewRepairEnabled() + { + config.global_settings.materialized_view_repair_enabled = true; + + boolean result = config.getMaterializedViewRepairEnabled(repairType); + + assertTrue(result); + } + + @Test + public void testSetMVRepairEnabled() + { + config.setMaterializedViewRepairEnabled(repairType, true); + + assertTrue(config.getOptions(repairType).materialized_view_repair_enabled); + } + + @Test + public void testSetForceRepairNewNode() + { + config.setForceRepairNewNode(repairType, true); + + assertTrue(config.getOptions(repairType).force_repair_new_node); + } + + @Test + public void testGetForceRepairNewNode() + { + config.global_settings.force_repair_new_node = true; + + boolean result = config.getForceRepairNewNode(repairType); + + assertTrue(result); + } + + @Test + public void testIsAutoRepairSchedulingEnabledDefault() + { + config = new AutoRepairConfig(); + + boolean result = config.isAutoRepairSchedulingEnabled(); + + assertFalse(result); + } + + @Test + public void testIsAutoRepairSchedulingEnabledTrue() + { + boolean result = config.isAutoRepairSchedulingEnabled(); + + assertTrue(result); + } + + @Test + public void testGetDefaultOptionsMVRepairIsEnabledByDefault() + { + Options defaultOptions = Options.getDefaultOptions(); + + assertFalse(defaultOptions.materialized_view_repair_enabled); + } + + @Test + public void testGetDefaultOptionsTokenRangeSplitter() + { + Options defaultOptions = Options.getDefaultOptions(); + + ParameterizedClass expectedDefault = new ParameterizedClass(RepairTokenRangeSplitter.class.getName(), Collections.emptyMap()); + + assertEquals(expectedDefault, defaultOptions.token_range_splitter); + assertEquals(RepairTokenRangeSplitter.class.getName(), AutoRepairConfig.newAutoRepairTokenRangeSplitter(repairType, defaultOptions.token_range_splitter).getClass().getName()); + } + + @Test(expected = ConfigurationException.class) + public void testInvalidTokenRangeSplitter() + { + AutoRepairConfig.newAutoRepairTokenRangeSplitter(repairType, new ParameterizedClass("invalid-class", Collections.emptyMap())); + } + + @Test + public void testSetInitialSchedulerDelay() + { + config.setInitialSchedulerDelay(repairType, "5s"); + + assertEquals(5, config.getOptions(repairType).initial_scheduler_delay.toSeconds()); + } + + @Test + public void testGetInitialSchedulerDelay() + { + config.global_settings.initial_scheduler_delay = new DurationSpec.IntSecondsBound("5s"); + + int result = config.getInitialSchedulerDelay(repairType).toSeconds(); + + assertEquals(5, result); + } + + @Test + public void testSetRepairSessionTimeout() + { + config.setRepairSessionTimeout(repairType, "1h"); + + assertEquals(3600, config.getOptions(repairType).repair_session_timeout.toSeconds()); + } + + @Test + public void testDefaultOptions() + { + Map defaultOptions = Options.getDefaultOptionsMap(); + Options options = defaultOptions.get(repairType); + assertFalse(options.enabled); + assertTrue(options.repair_by_keyspace); + assertEquals(Integer.valueOf(1), options.number_of_repair_threads); + assertEquals(Integer.valueOf(3), options.parallel_repair_count); + assertEquals(Integer.valueOf(3), options.parallel_repair_percentage); + assertEquals(Integer.valueOf(50000), options.sstable_upper_threshold); + assertEquals(new HashSet<>(), options.ignore_dcs); + assertTrue(options.repair_primary_token_range_only); + assertFalse(options.force_repair_new_node); + assertEquals(new DurationSpec.IntSecondsBound("6h"), options.table_max_repair_time); + assertFalse(options.materialized_view_repair_enabled); + assertEquals(new ParameterizedClass(RepairTokenRangeSplitter.class.getName(), Collections.emptyMap()), options.token_range_splitter); + assertEquals(new DurationSpec.IntSecondsBound("5m"), options.initial_scheduler_delay); + assertEquals(new DurationSpec.IntSecondsBound("3h"), options.repair_session_timeout); + assertEquals(new DurationSpec.IntSecondsBound("24h"), options.min_repair_interval); + } + + @Test + public void testGlobalOptions() + { + AutoRepairConfig config = new AutoRepairConfig(); + assertFalse(config.global_settings.enabled); + assertTrue(config.global_settings.repair_by_keyspace); + assertEquals(Integer.valueOf(1), config.global_settings.number_of_repair_threads); + assertEquals(Integer.valueOf(3), config.global_settings.parallel_repair_count); + assertEquals(Integer.valueOf(3), config.global_settings.parallel_repair_percentage); + assertEquals(Integer.valueOf(50000), config.global_settings.sstable_upper_threshold); + assertEquals(new HashSet<>(), config.global_settings.ignore_dcs); + assertTrue(config.global_settings.repair_primary_token_range_only); + assertFalse(config.global_settings.force_repair_new_node); + assertEquals(new DurationSpec.IntSecondsBound("6h"), config.global_settings.table_max_repair_time); + assertFalse(config.global_settings.materialized_view_repair_enabled); + assertEquals(new ParameterizedClass(RepairTokenRangeSplitter.class.getName(), Collections.emptyMap()), config.global_settings.token_range_splitter); + assertEquals(new DurationSpec.IntSecondsBound("5m"), config.global_settings.initial_scheduler_delay); + assertEquals(new DurationSpec.IntSecondsBound("3h"), config.global_settings.repair_session_timeout); + assertEquals(new DurationSpec.IntSecondsBound("24h"), config.global_settings.min_repair_interval); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java new file mode 100644 index 000000000000..fecb81d68d08 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; + +import com.google.common.collect.ImmutableSet; + +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.SystemDistributedKeyspace; +import org.apache.cassandra.schema.TableMetadata; + +/** + * Unit tests for {@link org.apache.cassandra.schema.SystemDistributedKeyspace} + */ +public class AutoRepairKeyspaceTest +{ + private static final Set tables = ImmutableSet.of( + SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY + ); + + @BeforeClass + public static void setupDatabaseDescriptor() + { + // Enable auto-repair feature so the tables are included in metadata + CassandraRelevantProperties.AUTOREPAIR_ENABLE.setBoolean(true); + DatabaseDescriptor.daemonInitialization(); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + } + + @Test + public void testEnsureAutoRepairTablesArePresent() + { + KeyspaceMetadata keyspaceMetadata = SystemDistributedKeyspace.metadata(); + Iterator iter = keyspaceMetadata.tables.iterator(); + Set actualDistributedTablesIter = new HashSet<>(); + while (iter.hasNext()) + { + actualDistributedTablesIter.add(iter.next().name); + } + + Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); + Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairMetricsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairMetricsTest.java new file mode 100644 index 000000000000..d0b053a58778 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairMetricsTest.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.StorageService; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class AutoRepairMetricsTest extends CQLTester +{ + + private AutoRepairMetrics metrics; + + @BeforeClass + public static void setupClass() throws Exception + { + setAutoRepairEnabled(true); + requireNetwork(); + AutoRepairUtils.setup(); + StorageService.instance.doAutoRepairSetup(); + + // Set min repair interval to an hour. + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairMinInterval(RepairType.FULL, "1h"); + } + + @Before + public void setup() + { + metrics = AutoRepairMetricsManager.getMetrics(RepairType.FULL); + } + + @Test + public void testShouldRecordRepairStartLagAndResetOnMyTurn() + { + // record a last finish repair time of one day. + long oneDayAgo = AutoRepair.instance.currentTimeMs() - 86_400_000; + metrics.recordRepairStartLag(oneDayAgo); + + // expect a recorded lag time of approximately 1 day (last repair finish time) - 1 hour (min repair interval) + long expectedLag = 86400 - 3600; + long recordedLag = metrics.repairStartLagSec.getValue(); + assertTrue(String.format("Expected at last 23h of lag (%d) but got (%d)", expectedLag, recordedLag), + recordedLag >= expectedLag); + // Given timing, allow at most 5 seconds of skew. + assertTrue(String.format("Expected 23h of lag (%d) but got a larger value (%d)", expectedLag, recordedLag), + recordedLag <= expectedLag + 5); + + // expect lag time to be restarted when recording a turn. + metrics.recordTurn(RepairTurn.MY_TURN); + assertEquals(0, metrics.repairStartLagSec.getValue().intValue()); + } + + @Test + public void testShouldRecordRepairStartLagOfZeroWhenFinishTimeIsWithinMinRepairInterval() + { + // record a last finish repair time of one 30 minutes + long thirtyMinutesAgo = AutoRepair.instance.currentTimeMs() - 1_800_000; + metrics.recordRepairStartLag(thirtyMinutesAgo); + + // expect 0 lag because last repair finish time was less than min repair interval + assertEquals(0, metrics.repairStartLagSec.getValue().intValue()); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java new file mode 100644 index 000000000000..34c74156e269 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -0,0 +1,934 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; + +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.cql3.statements.schema.TableAttributes; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.locator.LocalStrategy; +import org.apache.cassandra.repair.RepairParallelism; +import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.repair.messages.RepairOption; +import org.apache.cassandra.schema.AutoRepairParams; +import org.apache.cassandra.schema.SystemDistributedKeyspace; +import org.apache.cassandra.schema.TableParams; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.concurrent.ScheduledExecutorPlus; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.progress.ProgressEvent; +import org.apache.cassandra.utils.progress.ProgressEventType; +import org.apache.cassandra.utils.progress.ProgressListener; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.mockito.invocation.InvocationOnMock; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link org.apache.cassandra.repair.autorepair.AutoRepair} + */ +@RunWith(Parameterized.class) +public class AutoRepairParameterizedTest extends CQLTester +{ + private static final String KEYSPACE = "ks"; + private static final String TABLE = "tbl"; + private static final String TABLE_DISABLED_AUTO_REPAIR = "tbl_disabled_auto_repair"; + private static final String MV = "mv"; + private static TableMetadata cfm; + private static TableMetadata cfmDisabledAutoRepair; + private static Keyspace keyspace; + private static int timeFuncCalls; + @Mock + ScheduledExecutorPlus mockExecutor; + @Mock + AutoRepairState autoRepairState; + @Mock + RepairCoordinator repairRunnable; + + // Expected number of repairs to be executed. + private static int expectedRepairAssignments; + + @Parameterized.Parameter() + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameters(name = "repairType={0}") + public static Collection repairTypes() + { + return Arrays.asList(AutoRepairConfig.RepairType.values()); + } + + @BeforeClass + public static void setupClass() throws Exception + { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + setAutoRepairEnabled(true); + requireNetwork(); + AutoRepairUtils.setup(); + + cfm = TableMetadata.builder(KEYSPACE, TABLE) + .addPartitionKeyColumn("k", UTF8Type.instance) + .addStaticColumn("s", UTF8Type.instance) + .addClusteringColumn("i", IntegerType.instance) + .addRegularColumn("v", UTF8Type.instance) + .params(TableParams.builder() + .automatedRepair(AutoRepairParams.create(ImmutableMap.of(AutoRepairParams.Option.FULL_ENABLED.name().toLowerCase(), Boolean.toString(true), + AutoRepairParams.Option.INCREMENTAL_ENABLED.name().toLowerCase(), Boolean.toString(true), + AutoRepairParams.Option.PREVIEW_REPAIRED_ENABLED.name().toLowerCase(), Boolean.toString(true), + AutoRepairParams.Option.PRIORITY.name().toString(), Integer.toString(0)))) + .build()) + .build(); + + cfmDisabledAutoRepair = TableMetadata.builder(KEYSPACE, TABLE_DISABLED_AUTO_REPAIR) + .addPartitionKeyColumn("k", UTF8Type.instance) + .addStaticColumn("s", UTF8Type.instance) + .addClusteringColumn("i", IntegerType.instance) + .addRegularColumn("v", UTF8Type.instance) + .params(TableParams.builder() + .automatedRepair(AutoRepairParams.create(ImmutableMap.of(AutoRepairParams.Option.FULL_ENABLED.name().toLowerCase(), Boolean.toString(false), + AutoRepairParams.Option.INCREMENTAL_ENABLED.name().toLowerCase(), Boolean.toString(false), + AutoRepairParams.Option.PREVIEW_REPAIRED_ENABLED.name().toLowerCase(), Boolean.toString(false), + AutoRepairParams.Option.PRIORITY.name().toString(), Integer.toString(0)))) + .build()) + .build(); + + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), cfm, cfmDisabledAutoRepair); + cfm = Schema.instance.getTableMetadata(KEYSPACE, TABLE); + cfmDisabledAutoRepair = Schema.instance.getTableMetadata(KEYSPACE, TABLE_DISABLED_AUTO_REPAIR); + keyspace = Keyspace.open(KEYSPACE); + DatabaseDescriptor.setMaterializedViewsEnabled(true); + QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); + + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); + + // Calculate the expected number of keyspaces to be repaired, this should be all system keyspaces that are + // distributed, plus 1 for the table we created (ks.tbl). + int expectedKeyspacesGoingThroughRepair = 0; + for (Keyspace keyspace : Keyspace.all()) + { + // skip LocalStrategy keyspaces as these aren't repaired. + if (keyspace.getReplicationStrategy() instanceof LocalStrategy) + { + continue; + } + // skip system_traces keyspaces + if (keyspace.getName().equalsIgnoreCase(SchemaConstants.TRACE_KEYSPACE_NAME)) + { + continue; + } + expectedKeyspacesGoingThroughRepair += 1; + } + // Since the splitter will unwrap a full token range, we expect twice as many repairs. + expectedRepairAssignments = expectedKeyspacesGoingThroughRepair * 2; + } + + @Before + public void setup() + { + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + //noinspection resource + MockitoAnnotations.initMocks(this); + + Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).truncateBlocking(); + Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).disableAutoCompaction(); + + Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).truncateBlocking(); + Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).disableAutoCompaction(); + + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY).truncateBlocking(); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_HISTORY).truncateBlocking(); + + AutoRepair.instance.isSetupDone = false; + AutoRepair.instance.setup(); + executeCQL(); + + timeFuncCalls = 0; + AutoRepair.timeFunc = System::currentTimeMillis; + AutoRepair.sleepFunc = (Long startTime, TimeUnit unit) -> {}; + resetCounters(); + resetConfig(); + + AutoRepair.shuffleFunc = java.util.Collections::shuffle; + } + + @After + public void tearDown() + { + System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); + System.clearProperty("cassandra.streaming.requires_cdc_replay"); + } + + private void resetCounters() + { + AutoRepairMetrics metrics = AutoRepairMetricsManager.getMetrics(repairType); + Metrics.removeMatching((name, metric) -> name.startsWith("repairTurn")); + metrics.repairTurnMyTurn = Metrics.counter(String.format("repairTurnMyTurn-%s", repairType)); + metrics.repairTurnMyTurnForceRepair = Metrics.counter(String.format("repairTurnMyTurnForceRepair-%s", repairType)); + metrics.repairTurnMyTurnDueToPriority = Metrics.counter(String.format("repairTurnMyTurnDueToPriority-%s", repairType)); + } + + private void resetConfig() + { + // prepare a fresh default config + AutoRepairConfig defaultConfig = new AutoRepairConfig(true); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + defaultConfig.setAutoRepairEnabled(repairType, true); + defaultConfig.setMaterializedViewRepairEnabled(repairType, false); + } + + // reset the AutoRepairService config to default + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.repair_type_overrides = defaultConfig.repair_type_overrides; + config.global_settings = defaultConfig.global_settings; + config.history_clear_delete_hosts_buffer_interval = defaultConfig.history_clear_delete_hosts_buffer_interval; + config.repair_task_min_duration = new DurationSpec.LongSecondsBound("0s"); + } + + private void executeCQL() + { + QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, s) VALUES ('k', 's')"); + QueryProcessor.executeInternal("SELECT s FROM ks.tbl WHERE k='k'"); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME) + .getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY) + .forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + } + + @Test + public void testRepairTurn() + { + UUID myId = Gossiper.instance.getHostId(FBUtilities.getBroadcastAddressAndPort()); + Assert.assertNotEquals("Expected my turn for the repair", NOT_MY_TURN, AutoRepairUtils.myTurnToRunRepair(repairType, myId)); + } + + @Test + public void testRepair() + { + AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repair(repairType); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + long lastRepairTime = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + //if repair was done then lastRepairTime should be non-zero + Assert.assertTrue(String.format("Expected lastRepairTime > 0, actual value lastRepairTime %d", + lastRepairTime), lastRepairTime > 0); + // repair start lag sec should be reset on a successful repair + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).repairStartLagSec.getValue().intValue()); + } + + @Test + public void testTooFrequentRepairs() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + //in the first round let repair run + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repair(repairType); + long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); + Assert.assertNotEquals(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), + 0, consideredTables); + + //if repair was done in last 24 hours then it should not trigger another repair + config.setRepairMinInterval(repairType, "24h"); + AutoRepair.instance.repair(repairType); + long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + Assert.assertEquals(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + + "lastRepairTime2 %d", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + } + + @Test + public void testNonFrequentRepairs() + { + Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + long prevCount = state.getTotalMVTablesConsideredForRepair(); + AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repair(repairType); + long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + Assert.assertTrue(String.format("Expected lastRepairTime1 > 0, actual value lastRepairTime1 %d", + lastRepairTime1), lastRepairTime1 > 0); + UUID myId = Gossiper.instance.getHostId(FBUtilities.getBroadcastAddressAndPort()); + Assert.assertNotEquals("Expected my turn for the repair", + NOT_MY_TURN, AutoRepairUtils.myTurnToRunRepair(repairType, myId)); + AutoRepair.instance.repair(repairType); + long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + Assert.assertNotSame(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + + "lastRepairTime2 %d", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); + assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); + assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + } + + @Test + public void testGetPriorityHosts() + { + Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + long prevCount = state.getTotalMVTablesConsideredForRepair(); + AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); + Assert.assertEquals(String.format("Priority host count is not same, actual value %d, expected value %d", + AutoRepairUtils.getPriorityHosts(repairType).size(), 0), 0, AutoRepairUtils.getPriorityHosts(repairType).size()); + UUID myId = Gossiper.instance.getHostId(FBUtilities.getBroadcastAddressAndPort()); + Assert.assertNotEquals("Expected my turn for the repair", NOT_MY_TURN, AutoRepairUtils.myTurnToRunRepair(repairType, myId)); + AutoRepair.instance.repair(repairType); + AutoRepairUtils.addPriorityHosts(repairType, Sets.newHashSet(FBUtilities.getBroadcastAddressAndPort())); + AutoRepair.instance.repair(repairType); + Assert.assertEquals(String.format("Priority host count is not same actual value %d, expected value %d", + AutoRepairUtils.getPriorityHosts(repairType).size(), 0), 0, AutoRepairUtils.getPriorityHosts(repairType).size()); + assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); + assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + } + + @Test + public void testCheckAutoRepairStartStop() throws Throwable + { + Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + long prevCount = state.getTotalMVTablesConsideredForRepair(); + config.setRepairMinInterval(repairType, "0s"); + config.setAutoRepairEnabled(repairType, false); + long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + AutoRepair.instance.repair(repairType); + long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + //Since repair has not happened, both the last repair times should be same + Assert.assertEquals(String.format("Expected lastRepairTime1 %d, and lastRepairTime2 %d to be same", + lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); + + config.setAutoRepairEnabled(repairType, true); + AutoRepair.instance.repair(repairType); + //since repair is done now, so lastRepairTime1/lastRepairTime2 and lastRepairTime3 should not be same + long lastRepairTime3 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + Assert.assertNotSame(String.format("Expected lastRepairTime1 %d, and lastRepairTime3 %d to be not same", + lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime3); + assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); + assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + } + + @Test + public void testRepairPrimaryRangesByDefault() + { + Assert.assertTrue("Expected primary range repair only", + AutoRepairService.instance.getAutoRepairConfig().getRepairPrimaryTokenRangeOnly(repairType)); + } + + @Test + public void testGetAllMVs() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setMaterializedViewRepairEnabled(repairType, false); + assertFalse(config.getMaterializedViewRepairEnabled(repairType)); + assertEquals(0, AutoRepairUtils.getAllMVs(repairType, keyspace, cfm).size()); + + config.setMaterializedViewRepairEnabled(repairType, true); + + assertTrue(config.getMaterializedViewRepairEnabled(repairType)); + assertEquals(Collections.singletonList(MV), AutoRepairUtils.getAllMVs(repairType, keyspace, cfm)); + config.setMaterializedViewRepairEnabled(repairType, false); + } + + + @Test + public void testMVRepair() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setMaterializedViewRepairEnabled(repairType, true); + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); + AutoRepair.instance.repair(repairType); + assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + + config.setMaterializedViewRepairEnabled(repairType, false); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); + AutoRepair.instance.repair(repairType); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + + config.setMaterializedViewRepairEnabled(repairType, true); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); + AutoRepair.instance.repair(repairType); + assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + } + + @Test + public void testSkipRepairSSTableCountHigherThreshold() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + ColumnFamilyStore cfsBaseTable = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); + ColumnFamilyStore cfsMVTable = Keyspace.open(KEYSPACE).getColumnFamilyStore(MV); + Set preBaseTable = cfsBaseTable.getLiveSSTables(); + Set preMVTable = cfsBaseTable.getLiveSSTables(); + config.setRepairMinInterval(repairType, "0s"); + + for (int i = 0; i < 10; i++) + { + QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (k, i, v) VALUES('k1', %d, 'v1')", KEYSPACE, TABLE, i)); + cfsBaseTable.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + cfsMVTable.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + } + + Set postBaseTable = cfsBaseTable.getLiveSSTables(); + Set diffBaseTable = new HashSet<>(postBaseTable); + diffBaseTable.removeAll(preBaseTable); + assert diffBaseTable.size() == 10; + + Set postMVTable = cfsBaseTable.getLiveSSTables(); + Set diffMVTable = new HashSet<>(postMVTable); + diffMVTable.removeAll(preMVTable); + assert diffMVTable.size() == 10; + + int beforeCount = config.getRepairSSTableCountHigherThreshold(repairType); + config.setMaterializedViewRepairEnabled(repairType, true); + config.setRepairSSTableCountHigherThreshold(repairType, 9); + assertEquals(0, state.getSkippedTokenRangesCount()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + state.setLastRepairTime(0); + AutoRepair.instance.repair(repairType); + assertEquals(0, state.getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + // skipping both the tables - one table is due to its repair has been disabled, and another one due to high sstable count + assertEquals(0, state.getSkippedTokenRangesCount()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(2, state.getSkippedTablesCount()); + assertEquals(2, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + + // set it to higher value, and this time, the tables should not be skipped + config.setRepairSSTableCountHigherThreshold(repairType, beforeCount); + state.setLastRepairTime(0); + state.setSkippedTablesCount(0); + state.setTotalMVTablesConsideredForRepair(0); + AutoRepair.instance.repair(repairType); + assertEquals(1, state.getTotalMVTablesConsideredForRepair()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertEquals(0, state.getSkippedTokenRangesCount()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(1, state.getSkippedTablesCount()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + } + + @Test + public void testGetRepairState() + { + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getRepairKeyspaceCount()); + + AutoRepairState state = AutoRepair.instance.getRepairState(repairType); + state.setRepairKeyspaceCount(100); + + assertEquals(100L, AutoRepair.instance.getRepairState(repairType).getRepairKeyspaceCount()); + } + + @Test + public void testMetrics() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setMaterializedViewRepairEnabled(repairType, true); + config.setRepairMinInterval(repairType, "0s"); + config.setRepairRetryBackoff(repairType, "0s"); + config.setAutoRepairTableMaxRepairTime(repairType, "0s"); + AutoRepair.timeFunc = () -> { + timeFuncCalls++; + return timeFuncCalls * 1000L; + }; + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(1000L); + + AutoRepair.instance.repair(repairType); + + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).nodeRepairTimeInSec.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).clusterRepairTimeInSec.getValue() > 0); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).repairTurnMyTurn.getCount()); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue() > 0); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue().intValue()); + + config.setAutoRepairTableMaxRepairTime(repairType, String.valueOf(Integer.MAX_VALUE-1) + 's'); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) + .thenReturn(repairRunnable); + doAnswer(invocation -> { + invocation.getArgument(0, ProgressListener.class).progress("test", new ProgressEvent(ProgressEventType.COMPLETE, 0, 0)); + return null; + }).when(repairRunnable).addProgressListener(any()); + when(autoRepairState.getFailedTokenRangesCount()).thenReturn(10); + when(autoRepairState.getSucceededTokenRangesCount()).thenReturn(11); + when(autoRepairState.getLongestUnrepairedSec()).thenReturn(10); + + AutoRepair.instance.repair(repairType); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).failedTokenRangesCount.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).succeededTokenRangesCount.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue() > 0); + } + + @Test + public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + when(autoRepairState.getLastRepairTime()).thenReturn((long) 0); + AtomicInteger getRepairRunnableCalls = new AtomicInteger(); + AtomicReference prevListener = new AtomicReference<>(); + doAnswer(invocation -> { + if (getRepairRunnableCalls.getAndIncrement() > 0) + { + // progress listener from previous repair should be signalled before starting new repair + assertTrue(prevListener.get().condition.isSignalled()); + } + getRepairRunnableCalls.incrementAndGet(); + return repairRunnable; + }).when(autoRepairState).getRepairRunnable(any(), any(), any(), anyBoolean()); + doAnswer(invocation -> { + // sending out a COMPLETE event with a 10ms delay + Executors.newScheduledThreadPool(1).schedule(() -> { + invocation.getArgument(0, AutoRepair.RepairProgressListener.class).progress("test", new ProgressEvent(ProgressEventType.COMPLETE, 0, 0)); + }, 10, TimeUnit.MILLISECONDS); + return null; + }).when(repairRunnable).addProgressListener(any()); + + AutoRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); + } + + @Test + public void testDisabledAutoRepairForATableThroughTableLevelConfiguration() + { + Assert.assertTrue(cfm.params.autoRepair.repairEnabled(AutoRepairConfig.RepairType.FULL)); + Assert.assertTrue(cfm.params.autoRepair.repairEnabled(AutoRepairConfig.RepairType.INCREMENTAL)); + Assert.assertFalse(cfmDisabledAutoRepair.params.autoRepair.repairEnabled(AutoRepairConfig.RepairType.FULL)); + Assert.assertFalse(cfmDisabledAutoRepair.params.autoRepair.repairEnabled(AutoRepairConfig.RepairType.INCREMENTAL)); + + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairMinInterval(repairType, "0s"); + int disabledTablesRepairCountBefore = AutoRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); + AutoRepair.instance.repair(repairType); + int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); + Assert.assertNotSame(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), + 0, consideredTables); + int disabledTablesRepairCountAfter = AutoRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); + Assert.assertTrue(String.format("A table %s should be skipped from auto repair, expected value: %d, actual value %d ", TABLE_DISABLED_AUTO_REPAIR, disabledTablesRepairCountBefore + 1, disabledTablesRepairCountAfter), + disabledTablesRepairCountBefore < disabledTablesRepairCountAfter); + } + + @Test + public void testTableAttribute() + { + assertTrue(TableAttributes.validKeywords().contains("auto_repair")); + } + + @Test + public void testDefaultAutomatedRepair() + { + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + Assert.assertTrue(String.format("expected repair type %s to be enabled on table %s", repairType, cfm.name), + cfm.params.autoRepair.repairEnabled(repairType)); + Assert.assertFalse(String.format("expected repair type %s to be disabled on table %s", repairType, cfmDisabledAutoRepair.name), + cfmDisabledAutoRepair.params.autoRepair.repairEnabled(repairType)); + } + } + + @Test + public void testRepairShufflesKeyspacesAndTables() + { + AtomicInteger shuffleKeyspacesCall = new AtomicInteger(); + AtomicInteger shuffleTablesCall = new AtomicInteger(); + AtomicInteger keyspaceCount = new AtomicInteger(); + AutoRepair.shuffleFunc = (List list) -> { + // check whether was invoked for keyspaces or tables + if (list.contains(KEYSPACE)) + { + shuffleKeyspacesCall.getAndIncrement(); + keyspaceCount.set(list.size()); + } + else + // presume list not containing a keyspace is for tables. + shuffleTablesCall.getAndIncrement(); + }; + + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repair(repairType); + + // Expect a single invocation for keyspaces + assertEquals(1, shuffleKeyspacesCall.get()); + // Expect an invocation for tables for each keyspace + assertNotEquals(0, keyspaceCount.get()); + assertEquals(keyspaceCount.get(), shuffleTablesCall.get()); + } + + @Test + public void testRepairTakesLastRepairTimeFromDB() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setMaterializedViewRepairEnabled(repairType, true); + long lastRepairTime = System.currentTimeMillis() - 1000; + AutoRepairUtils.insertNewRepairHistory(repairType, 0, lastRepairTime); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + config.setRepairMinInterval(repairType, "1h"); + + AutoRepair.instance.repair(repairType); + + // repair scheduler should not attempt to run repair as last repair time in DB is current time - 1s + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair()); + // repair scheduler should load the repair time from the DB + assertEquals(lastRepairTime, AutoRepair.instance.repairStates.get(repairType).getLastRepairTime()); + } + + @Test + public void testRepairMaxRetries() + { + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + doAnswer(invocation -> { + invocation.getArgument(0, ProgressListener.class).progress("test", new ProgressEvent(ProgressEventType.ERROR, 0, 0)); + return null; + }).when(repairRunnable).addProgressListener(any()); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AtomicInteger sleepCalls = new AtomicInteger(); + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + sleepCalls.getAndIncrement(); + assertEquals(TimeUnit.SECONDS, unit); + assertEquals(config.getRepairRetryBackoff(repairType).toSeconds(), (long) duration); + }; + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + + AutoRepair.instance.repair(repairType); + + // Expect configured retries for each keyspace expected to be repaired + assertEquals(config.getRepairMaxRetries(repairType)*expectedRepairAssignments, sleepCalls.get()); + verify(autoRepairState, times(1)).setSucceededTokenRangesCount(0); + verify(autoRepairState, times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, times(1)).setFailedTokenRangesCount(expectedRepairAssignments); + } + + @Test + public void testRepairSuccessAfterRetry() + { + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AtomicInteger sleepCalls = new AtomicInteger(); + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + sleepCalls.getAndIncrement(); + assertEquals(TimeUnit.SECONDS, unit); + assertEquals(config.getRepairRetryBackoff(repairType).toSeconds(), (long) duration); + }; + doAnswer(invocation -> { + if (sleepCalls.get() == 0) + { + invocation.getArgument(0, ProgressListener.class).progress("test", new ProgressEvent(ProgressEventType.ERROR, 0, 0)); + } + else + { + invocation.getArgument(0, ProgressListener.class).progress("test", new ProgressEvent(ProgressEventType.COMPLETE, 0, 0)); + } + + return null; + }).when(repairRunnable).addProgressListener(any()); + config.setRepairMinInterval(repairType, "0s"); + config.setRepairMaxRetries(repairType, 1); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + AutoRepair.instance.repair(repairType); + + assertEquals(1, sleepCalls.get()); + verify(autoRepairState, times(1)).setSucceededTokenRangesCount(expectedRepairAssignments); + verify(autoRepairState, times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, times(1)).setFailedTokenRangesCount(0); + } + + @Test + public void testRepairDoesNotThrowsForIRWithMVReplayButMVRepairDisabled() + { + AutoRepair.instance.setup(); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); + AutoRepairService.instance.getAutoRepairConfig().setMaterializedViewRepairEnabled(repairType, false); + + if (repairType == AutoRepairConfig.RepairType.INCREMENTAL) + { + try + { + AutoRepair.instance.repair(repairType); + } + catch (ConfigurationException ignored) + { + fail("ConfigurationException not expected"); + } + } + else + { + AutoRepair.instance.repair(repairType); + } + } + + @Test + public void testRepairThrowsForIRWithMVReplay() + { + AutoRepair.instance.setup(); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); + AutoRepairService.instance.getAutoRepairConfig().setMaterializedViewRepairEnabled(repairType, true); + + if (repairType == AutoRepairConfig.RepairType.INCREMENTAL) + { + try + { + AutoRepair.instance.repair(repairType); + fail("Expected ConfigurationException"); + } + catch (ConfigurationException ignored) + { + } + } + else + { + AutoRepair.instance.repair(repairType); + } + } + + @Test + public void testRepairThrowsForIRWithCDCReplay() + { + AutoRepair.instance.setup(); + DatabaseDescriptor.setCDCEnabled(true); + DatabaseDescriptor.setCDCOnRepairEnabled(true); + + if (repairType == AutoRepairConfig.RepairType.INCREMENTAL) + { + try + { + AutoRepair.instance.repair(repairType); + fail("Expected ConfigurationException"); + } + catch (ConfigurationException ignored) + { + } + } + else + { + AutoRepair.instance.repair(repairType); + } + } + + @Test + public void testSoakAfterImmediateRepair() + { + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + doAnswer(invocation -> { + invocation.getArgument(0, ProgressListener.class).progress("test", new ProgressEvent(ProgressEventType.COMPLETE, 0, 0)); + return null; + }).when(repairRunnable).addProgressListener(any()); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.repair_task_min_duration = new DurationSpec.LongSecondsBound("10s"); + AtomicInteger sleepCalls = new AtomicInteger(); + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + sleepCalls.getAndIncrement(); + assertEquals(TimeUnit.MILLISECONDS, unit); + assertTrue(config.getRepairTaskMinDuration().toMilliseconds() >= duration); + config.repair_task_min_duration = new DurationSpec.LongSecondsBound("0s"); + }; + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + + AutoRepair.instance.repair(repairType); + + assertEquals(1, sleepCalls.get()); + verify(autoRepairState, times(1)).setSucceededTokenRangesCount(expectedRepairAssignments); + verify(autoRepairState, times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, times(1)).setFailedTokenRangesCount(0); + } + + @Test + public void testNoSoakAfterRepair() + { + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + doAnswer(invocation -> { + invocation.getArgument(0, ProgressListener.class).progress("test", new ProgressEvent(ProgressEventType.COMPLETE, 0, 0)); + return null; + }).when(repairRunnable).addProgressListener(any()); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.repair_task_min_duration = new DurationSpec.LongSecondsBound("0s"); + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + fail("Should not sleep after repair"); + }; + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + + AutoRepair.instance.repair(repairType); + + verify(autoRepairState, times(1)).setSucceededTokenRangesCount(expectedRepairAssignments); + verify(autoRepairState, times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, times(1)).setFailedTokenRangesCount(0); + } + + @Test + public void testSchedulerIgnoresErrorsFromUnrelatedRepairRunables() + { + RepairOption options = new RepairOption(RepairParallelism.PARALLEL, true, repairType == AutoRepairConfig.RepairType.INCREMENTAL, false, + AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), Collections.emptySet(), + false, false, false, PreviewKind.NONE, false, true, false, false); + AutoRepairState repairState = AutoRepair.instance.repairStates.get(repairType); + AutoRepairState spyState = spy(repairState); + AtomicReference failingListener = new AtomicReference<>(); + AtomicInteger repairRunableCalls = new AtomicInteger(); + doAnswer((InvocationOnMock inv ) -> { + RepairCoordinator runnable = spy(repairState.getRepairRunnable(inv.getArgument(0), inv.getArgument(1), inv.getArgument(2), + inv.getArgument(3))); + if (repairRunableCalls.getAndIncrement() == 0) + { + // this will be used for first repair job + doAnswer(invocation -> { + // repair runnable for the first repair job will immediately fail + failingListener.set(invocation.getArgument(0, AutoRepair.RepairProgressListener.class)); + invocation.getArgument(0, ProgressListener.class).progress("test", new ProgressEvent(ProgressEventType.ERROR, 0, 0)); + return null; + }).when(runnable).addProgressListener(any()); + } + else + { + // this will be used for subsequent repair jobs + doAnswer(invocation -> { + if (repairRunableCalls.get() > 0) + { + // repair runnable for the subsequent repair jobs will immediately complete + invocation.getArgument(0, ProgressListener.class).progress("test", new ProgressEvent(ProgressEventType.COMPLETE, 0, 0)); + + } + // repair runnable for the first repair job will continue firing ERROR events + failingListener.get().progress("test", new ProgressEvent(ProgressEventType.ERROR, 0, 0)); + return null; + }).when(runnable).addProgressListener(any()); + } + return runnable; + }).when(spyState).getRepairRunnable(any(), any(), any(), anyBoolean()); + when(spyState.getLastRepairTime()).thenReturn((long) 0); + AutoRepairService.instance.getAutoRepairConfig().setRepairMaxRetries(repairType, 0); + AutoRepair.instance.repairStates.put(repairType, spyState); + + AutoRepair.instance.repair(repairType); + + assertEquals(1, (int) AutoRepairMetricsManager.getMetrics(repairType).failedTokenRangesCount.getValue()); + // only the first repair job should have failed despite it continuously firing ERROR events + verify(spyState, times(1)).setFailedTokenRangesCount(1); + } + + @Test + public void testProgressError() + { + AutoRepair.RepairProgressListener listener = new AutoRepair.RepairProgressListener(repairType); + + listener.progress("test", new ProgressEvent(ProgressEventType.ERROR, 0, 0, "test")); + + assertFalse(listener.success); + assertTrue(listener.condition.isSignalled()); + } + + @Test + public void testProgressProgress() + { + AutoRepair.RepairProgressListener listener = new AutoRepair.RepairProgressListener(repairType); + + listener.progress("test", new ProgressEvent(ProgressEventType.PROGRESS, 0, 0, "test")); + + assertFalse(listener.success); + assertFalse(listener.condition.isSignalled()); + } + + @Test + public void testProgresComplete() + { + AutoRepair.RepairProgressListener listener = new AutoRepair.RepairProgressListener(repairType); + + listener.progress("test", new ProgressEvent(ProgressEventType.COMPLETE, 0, 0, "test")); + + assertTrue(listener.success); + assertTrue(listener.condition.isSignalled()); + } + + @Test + public void testAwait() throws Exception + { + AutoRepair.RepairProgressListener listener = new AutoRepair.RepairProgressListener(repairType); + listener.progress("test", new ProgressEvent(ProgressEventType.COMPLETE, 0, 0, "test")); + + listener.await(new DurationSpec.IntSecondsBound("12h")); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairShutdownTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairShutdownTest.java new file mode 100644 index 000000000000..c7de17e0e455 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairShutdownTest.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Unit tests to validate the executor shutdown inside {@link AutoRepair} + */ +public class AutoRepairShutdownTest extends CQLTester +{ + @BeforeClass + public static void setupClass() throws Exception + { + setAutoRepairEnabled(true); + requireNetwork(); + } + + @Test + public void testAutoRepairShutdown() throws Exception + { + AutoRepair.instance.setup(); + + for (RepairType type : RepairType.values()) + { + assertFalse("RepairRunnableExecutor should not have been shut down", AutoRepair.instance.getRepairRunnableExecutors().get(type).isShutdown()); + assertFalse("RepairExecutor should not have been shut down", AutoRepair.instance.getRepairExecutors().get(type).isShutdown()); + } + assertFalse("AutoRepair should not be marked as shut down", AutoRepair.instance.isShutDown); + + AutoRepair.instance.shutdownBlocking(); + + for (RepairType type : RepairType.values()) + { + assertTrue("RepairRunnableExecutor should be shut down", AutoRepair.instance.getRepairRunnableExecutors().get(type).isShutdown()); + assertTrue("RepairExecutor should be shut down", AutoRepair.instance.getRepairExecutors().get(type).isShutdown()); + } + assertTrue("AutoRepair should be marked as shut down", AutoRepair.instance.isShutDown); + + try + { + AutoRepair.instance.shutdownBlocking(); + fail("A second call to shutdown should have thrown an exception"); + } + catch (IllegalStateException e) + { + // expected + } + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java new file mode 100644 index 000000000000..fe2d542961dc --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import org.junit.Test; + +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +/** + * Unit tests for {@link org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType} + */ +public class AutoRepairStateFactoryTest +{ + @Test + public void testGetRepairState() + { + AutoRepairState state = RepairType.getAutoRepairState(RepairType.FULL, new AutoRepairConfig()); + + assertNotNull(state); + assertTrue(state.getClass().getSimpleName().equals("FullRepairState")); + + state = RepairType.getAutoRepairState(RepairType.INCREMENTAL, new AutoRepairConfig()); + + assertNotNull(state); + assertTrue(state.getClass().getSimpleName().equals("IncrementalRepairState")); + + state = RepairType.getAutoRepairState(RepairType.PREVIEW_REPAIRED, new AutoRepairConfig()); + + assertNotNull(state); + assertTrue(state.getClass().getSimpleName().equals("PreviewRepairedState")); + } + + @Test + public void testGetRepairStateSupportsAllRepairTypes() + { + for (RepairType repairType : RepairType.values()) + { + try + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + assertNotNull(state); + } catch (IllegalArgumentException e) + { + assertNull(e); + } + } + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java new file mode 100644 index 000000000000..240a2db46fc9 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; +import org.apache.cassandra.service.AutoRepairService; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.MockitoAnnotations.initMocks; + +/** + * Unit tests for {@link org.apache.cassandra.repair.autorepair.AutoRepairState} + */ +@RunWith(Parameterized.class) +public class AutoRepairStateTest extends CQLTester +{ + private static final String testTable = "test"; + + @Parameterized.Parameter + public RepairType repairType; + + @Parameterized.Parameters + public static Collection repairTypes() + { + return Arrays.asList(RepairType.values()); + } + + @Before + public void setUp() + { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + initMocks(this); + createTable(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int)", KEYSPACE, testTable)); + } + + @Test + public void testGetRepairRunnable() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + AutoRepairService.setup(); + + Runnable runnable = state.getRepairRunnable(KEYSPACE, ImmutableList.of(testTable), ImmutableSet.of(), false); + + assertNotNull(runnable); + } + + @Test + public void testGetLastRepairTime() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.lastRepairTimeInMs = 1; + + assertEquals(1, state.getLastRepairTime()); + } + + @Test + public void testSetTotalTablesConsideredForRepair() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setTotalTablesConsideredForRepair(1); + + assertEquals(1, state.totalTablesConsideredForRepair); + } + + @Test + public void testGetTotalTablesConsideredForRepair() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.totalTablesConsideredForRepair = 1; + + assertEquals(1, state.getTotalTablesConsideredForRepair()); + } + + @Test + public void testSetLastRepairTimeInMs() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setLastRepairTime(1); + + assertEquals(1, state.lastRepairTimeInMs); + } + + @Test + public void testGetClusterRepairTimeInSec() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.clusterRepairTimeInSec = 1; + + assertEquals(1, state.getClusterRepairTimeInSec()); + } + + @Test + public void testGetNodeRepairTimeInSec() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.nodeRepairTimeInSec = 1; + + assertEquals(1, state.getNodeRepairTimeInSec()); + } + + @Test + public void testSetRepairInProgress() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setRepairInProgress(true); + + assertTrue(state.repairInProgress); + } + + @Test + public void testIsRepairInProgress() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.repairInProgress = true; + + assertTrue(state.isRepairInProgress()); + } + + @Test + public void testSetSkippedTokenRangesCount() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setSkippedTokenRangesCount(1); + + assertEquals(1, state.skippedTokenRangesCount); + } + + @Test + public void testGetSkippedTokenRangesCount() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.skippedTokenRangesCount = 1; + + assertEquals(1, state.getSkippedTokenRangesCount()); + } + + @Test + public void testGetLongestUnrepairedSecNull() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.longestUnrepairedNode = null; + + try + { + assertEquals(0, state.getLongestUnrepairedSec()); + } + catch (Exception e) + { + assertNull(e); + } + } + + @Test + public void testGetLongestUnrepairedSec() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.longestUnrepairedNode = new AutoRepairHistory(UUID.randomUUID(), "", 0, 1000, + null, 0, false); + AutoRepairState.timeFunc = () -> 2000L; + + try + { + assertEquals(1, state.getLongestUnrepairedSec()); + } + catch (Exception e) + { + assertNull(e); + } + } + + @Test + public void testSetTotalMVTablesConsideredForRepair() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setTotalMVTablesConsideredForRepair(1); + + assertEquals(1, state.totalMVTablesConsideredForRepair); + } + + @Test + public void testGetTotalMVTablesConsideredForRepair() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.totalMVTablesConsideredForRepair = 1; + + assertEquals(1, state.getTotalMVTablesConsideredForRepair()); + } + + @Test + public void testSetNodeRepairTimeInSec() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setNodeRepairTimeInSec(1); + + assertEquals(1, state.nodeRepairTimeInSec); + } + + @Test + public void testSetClusterRepairTimeInSec() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setClusterRepairTimeInSec(1); + + assertEquals(1, state.clusterRepairTimeInSec); + } + + @Test + public void testSetRepairKeyspaceCount() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setRepairKeyspaceCount(1); + + assertEquals(1, state.repairKeyspaceCount); + } + + @Test + public void testGetRepairKeyspaceCount() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.repairKeyspaceCount = 1; + + assertEquals(1, state.getRepairKeyspaceCount()); + } + + @Test + public void testSetLongestUnrepairedNode() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); + + state.setLongestUnrepairedNode(history); + + assertEquals(history, state.longestUnrepairedNode); + } + + @Test + public void testSetSucceededTokenRangesCount() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setSucceededTokenRangesCount(1); + + assertEquals(1, state.succeededTokenRangesCount); + } + + @Test + public void testGetSucceededTokenRangesCount() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.succeededTokenRangesCount = 1; + + assertEquals(1, state.getSucceededTokenRangesCount()); + } + + @Test + public void testSetFailedTokenRangesCount() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + + state.setFailedTokenRangesCount(1); + + assertEquals(1, state.failedTokenRangesCount); + } + + @Test + public void testGetFailedTokenRangesCount() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType, new AutoRepairConfig()); + state.failedTokenRangesCount = 1; + + assertEquals(1, state.getFailedTokenRangesCount()); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java new file mode 100644 index 000000000000..53eaca86dcbc --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.HashMap; +import java.util.Map; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.Assert; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.ReplicationParams; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.schema.SchemaTestUtil; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Unit tests for {@link org.apache.cassandra.repair.autorepair.AutoRepair} + */ +public class AutoRepairTest extends CQLTester +{ + @BeforeClass + public static void setupClass() throws Exception + { + setAutoRepairEnabled(true); + requireNetwork(); + } + + @Before + public void setup() + { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.FULL, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.INCREMENTAL, true); + } + + @Test + public void testSetup() + { + AutoRepair.instance.setup(); + assertEquals(RepairType.values().length, AutoRepair.instance.repairExecutors.size()); + for (RepairType repairType : AutoRepair.instance.repairExecutors.keySet()) + { + int expectedTasks = AutoRepair.instance.repairExecutors.get(repairType).getPendingTaskCount() + + AutoRepair.instance.repairExecutors.get(repairType).getActiveTaskCount(); + assertTrue(String.format("Expected > 0 task in queue for %s but was %s", repairType, expectedTasks), + expectedTasks > 0); + } + } + + @Test + public void testSafeGuardSetupCall() + { + // only one should be setup, and rest should be ignored + AutoRepair.instance.setup(); + AutoRepair.instance.setup(); + AutoRepair.instance.setup(); + + assertEquals(RepairType.values().length, AutoRepair.instance.repairExecutors.size()); + for (RepairType repairType : AutoRepair.instance.repairExecutors.keySet()) + { + int expectedTasks = AutoRepair.instance.repairExecutors.get(repairType).getPendingTaskCount() + + AutoRepair.instance.repairExecutors.get(repairType).getActiveTaskCount(); + assertTrue(String.format("Expected > 0 task in queue for %s but was %s", repairType, expectedTasks), + expectedTasks > 0); + } + } + + @Test(expected = ConfigurationException.class) + public void testSetupFailsWhenIREnabledWithCDCReplay() + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.INCREMENTAL, true); + DatabaseDescriptor.setCDCOnRepairEnabled(true); + DatabaseDescriptor.setCDCEnabled(true); + + AutoRepair.instance.isSetupDone = false; + AutoRepair.instance.setup(); + } + + @Test + public void testNoFailureIfMVRepairOnButConfigIsOff() + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.INCREMENTAL, true); + DatabaseDescriptor.getAutoRepairConfig().setMaterializedViewRepairEnabled(RepairType.INCREMENTAL, false); + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); + AutoRepair.instance.setup(); + } + + @Test(expected = ConfigurationException.class) + public void testSetupFailsWhenIREnabledWithMVReplay() + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.INCREMENTAL, true); + DatabaseDescriptor.getAutoRepairConfig().setMaterializedViewRepairEnabled(RepairType.INCREMENTAL, true); + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); + AutoRepair.instance.isSetupDone = false; + AutoRepair.instance.setup(); + } + + @Test + public void testCheckNTSreplicationNodeInsideOutsideDC() + { + String ksname1 = "ks_nts1"; + String ksname2 = "ks_nts2"; + Map configOptions1 = new HashMap<>(); + configOptions1.put("datacenter1", "3"); + configOptions1.put(ReplicationParams.CLASS, "NetworkTopologyStrategy"); + SchemaTestUtil.addOrUpdateKeyspace(KeyspaceMetadata.create(ksname1, KeyspaceParams.create(false, configOptions1)), false); + Map configOptions2 = new HashMap<>(); + configOptions2.put("datacenter2", "3"); + configOptions2.put(ReplicationParams.CLASS, "NetworkTopologyStrategy"); + SchemaTestUtil.addOrUpdateKeyspace(KeyspaceMetadata.create(ksname2, KeyspaceParams.create(false, configOptions2)), false); + + for (Keyspace ks : Keyspace.all()) + { + if (ks.getName().equals(ksname1)) + { + // case 1 : + // node reside in "datacenter1" + // keyspace has replica in "datacenter1" + Assert.assertTrue(AutoRepairUtils.shouldConsiderKeyspace(ks)); + } + else if (ks.getName().equals(ksname2)) + { + // case 2 : + // node reside in "datacenter1" + // keyspace has replica in "datacenter2" + Assert.assertFalse(AutoRepairUtils.shouldConsiderKeyspace(ks)); + } + } + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java new file mode 100644 index 000000000000..55c352c8318c --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -0,0 +1,715 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.net.UnknownHostException; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.UUID; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.EndpointState; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.gms.HeartBeatState; +import org.apache.cassandra.gms.VersionedValue; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.locator.IEndpointSnitch; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.CurrentRepairStatus; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SystemDistributedKeyspace; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.FBUtilities; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import org.apache.cassandra.config.CassandraRelevantProperties; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_DELETE_HOSTS; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_FORCE_REPAIR; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_FINISH_TS; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_PRIORITY; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_START_TS; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_TURN; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link org.apache.cassandra.repair.autorepair.AutoRepairUtils} + */ +public class AutoRepairUtilsTest extends CQLTester +{ + static final RepairType repairType = RepairType.INCREMENTAL; + static UUID hostId; + + static InetAddressAndPort localEndpoint; + + @Mock + static IEndpointSnitch snitchMock; + + static IEndpointSnitch defaultSnitch; + + @BeforeClass + public static void setupClass() throws Exception + { + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + CassandraRelevantProperties.AUTOREPAIR_ENABLE.setBoolean(true); + setAutoRepairEnabled(true); + requireNetwork(); + defaultSnitch = DatabaseDescriptor.getEndpointSnitch(); + localEndpoint = FBUtilities.getBroadcastAddressAndPort(); + hostId = Gossiper.instance.getHostId(localEndpoint); + AutoRepairUtils.setup(); + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace("ks", KeyspaceParams.create(false, + ImmutableMap.of("class", "NetworkTopologyStrategy", "datacenter1", "1")), + TableMetadata.builder("ks", "tbl") + .addPartitionKeyColumn("k", UTF8Type.instance) + .build()); + } + + @Before + public void setup() + { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + MockitoAnnotations.initMocks(this); + DatabaseDescriptor.setEndpointSnitch(defaultSnitch); + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); + // Clean up Gossiper state before each test - remove all non-local endpoints + Gossiper.instance.endpointStateMap.keySet().removeIf(ep -> !ep.equals(localEndpoint)); + Gossiper.instance.liveEndpoints.removeIf(ep -> !ep.equals(localEndpoint)); + } + + @Test + public void testSetForceRepair() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + AutoRepairUtils.setForceRepair(repairType, ImmutableSet.of(localEndpoint)); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + assertTrue(result.one().getBoolean(COL_FORCE_REPAIR)); + } + + @Test + public void testSetForceRepairNewNode() + { + AutoRepairUtils.setForceRepairNewNode(repairType); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + assertTrue(result.one().getBoolean(COL_FORCE_REPAIR)); + } + + @Test + public void testClearDeleteHosts() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, delete_hosts, delete_hosts_update_time) VALUES ('%s', %s, { %s }, toTimestamp(now()))", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId, hostId)); + + AutoRepairUtils.clearDeleteHosts(repairType, hostId); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT delete_hosts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + Set deleteHosts = result.one().getSet(COL_DELETE_HOSTS, UUIDType.instance); + assertNull(deleteHosts); + } + + @Test + public void testGetAutoRepairHistoryForLocalGroup() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + List history = AutoRepairUtils.getAutoRepairHistory(repairType); + assertNotNull(history); + assertEquals(1, history.size()); + assertEquals(hostId, history.get(0).hostId); + } + + @Test + public void testGetAutoRepairHistoryForLocalGroup_empty_history() + { + List history = AutoRepairUtils.getAutoRepairHistory(repairType); + + assertNull(history); + } + + @Test + public void testGetCurrentRepairStatus() + { + UUID forceRepair = UUID.randomUUID(); + UUID regularRepair = UUID.randomUUID(); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, force_repair, repair_start_ts) VALUES ('%s', %s, true, toTimestamp(now()))", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), forceRepair)); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, repair_start_ts) VALUES ('%s', %s, toTimestamp(now()))", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), regularRepair)); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString(), regularRepair)); + + CurrentRepairStatus status = AutoRepairUtils.getCurrentRepairStatus(repairType, AutoRepairUtils.getAutoRepairHistory(repairType), hostId); + + assertNotNull(status); + assertEquals(1, status.historiesWithoutOnGoingRepair.size()); + assertEquals(hostId, status.historiesWithoutOnGoingRepair.get(0).hostId); + assertEquals(1, status.hostIdsWithOnGoingRepair.size()); + assertTrue(status.hostIdsWithOnGoingRepair.contains(regularRepair)); + assertEquals(1, status.hostIdsWithOnGoingForceRepair.size()); + assertTrue(status.hostIdsWithOnGoingForceRepair.contains(forceRepair)); + assertEquals(1, status.priority.size()); + assertTrue(status.priority.contains(regularRepair)); + assertEquals(hostId, status.myRepairHistory.hostId); + } + + @Test + public void testGetHostIdsInCurrentRing() + { + TreeSet hosts = AutoRepairUtils.getHostIdsInCurrentRing(repairType); + + assertNotNull(hosts); + assertEquals(1, hosts.size()); + assertTrue(hosts.contains(hostId)); + } + + @Test + public void testGetHostIdsInCurrentRing_multiple_nodes() + { + InetAddressAndPort ignoredEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 1); + InetAddressAndPort deadEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 2); + DatabaseDescriptor.getAutoRepairConfig().setIgnoreDCs(repairType, ImmutableSet.of("dc2")); + DatabaseDescriptor.setEndpointSnitch(snitchMock); + when(snitchMock.getDatacenter(localEndpoint)).thenReturn("dc1"); + when(snitchMock.getDatacenter(ignoredEndpoint)).thenReturn("dc2"); + when(snitchMock.getDatacenter(deadEndpoint)).thenReturn("dc1"); + + TreeSet hosts = AutoRepairUtils.getHostIdsInCurrentRing(repairType, ImmutableSet.of(localEndpoint, ignoredEndpoint, deadEndpoint)); + + assertNotNull(hosts); + assertEquals(1, hosts.size()); + assertTrue(hosts.contains(hostId)); + } + + @Test + public void testHasMultipleLiveMajorVersionsWithSingleNode() + { + boolean result = AutoRepairUtils.hasMultipleLiveMajorVersions(); + assertFalse(result); + } + + @Test + public void testHasMultipleLiveMajorVersionsWithMultipleNodesOfSameVersion() throws UnknownHostException + { + Gossiper.instance.start(0); + Gossiper.instance.expireUpgradeFromVersion(); + + VersionedValue.VersionedValueFactory factory = new VersionedValue.VersionedValueFactory(null); + EndpointState es = new EndpointState((HeartBeatState) null); + es.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("4.1")); + + // Set local endpoint version to match test scenario + EndpointState localEs = Gossiper.instance.getEndpointStateForEndpoint(localEndpoint); + if (localEs != null) + { + localEs.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("4.1")); + } + + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.100"), es); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.100")); + + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.200"), es); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.200")); + + + boolean result = AutoRepairUtils.hasMultipleLiveMajorVersions(); + assertFalse(result); + } + + + @Test + public void testHasMultipleLiveMajorVersionsWithMultipleNodesOfSameMajorVersionDifferentMinorVersions() throws UnknownHostException + { + Gossiper.instance.start(0); + Gossiper.instance.expireUpgradeFromVersion(); + + VersionedValue.VersionedValueFactory factory = new VersionedValue.VersionedValueFactory(null); + EndpointState es = new EndpointState((HeartBeatState) null); + es.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("4.1")); + + // Set local endpoint version to match test scenario (same major version) + EndpointState localEs = Gossiper.instance.getEndpointStateForEndpoint(localEndpoint); + if (localEs != null) + { + localEs.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("4.0")); + } + + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.100"), es); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.100")); + + EndpointState es1 = new EndpointState((HeartBeatState) null); + es1.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("4.2")); + + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.200"), es1); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.200")); + + boolean result = AutoRepairUtils.hasMultipleLiveMajorVersions(); + assertFalse(result); + } + + @Test + public void testHasMultipleLiveMajorVersionsWithMultipleNodesOfDifferentMajorVersions() throws UnknownHostException + { + Gossiper.instance.start(0); + Gossiper.instance.expireUpgradeFromVersion(); + + VersionedValue.VersionedValueFactory factory = new VersionedValue.VersionedValueFactory(null); + EndpointState es = new EndpointState((HeartBeatState) null); + es.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("4.1")); + + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.100"), es); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.100")); + + EndpointState es1 = new EndpointState((HeartBeatState) null); + es1.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("5.1")); + + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.200"), es1); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.200")); + + boolean result = AutoRepairUtils.hasMultipleLiveMajorVersions(); + assertTrue(result); + } + + @Test + public void testGetHostWithLongestUnrepairTime() + { + UUID otherHostId = UUID.randomUUID(); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, repair_finish_ts) VALUES ('%s', %s, toTimestamp(now()))", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), otherHostId)); + + AutoRepairHistory history = AutoRepairUtils.getHostWithLongestUnrepairTime(repairType); + + assertEquals(hostId, history.hostId); + } + + @Test + public void testGetMaxNumberOfNodeRunAutoRepairInGroup_0_group_size() + { + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 2); + + int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepair(repairType, 0); + + assertEquals(2, count); + } + + @Test + public void testGetMaxNumberOfNodeRunAutoRepairInGroup_percentage() + { + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 2); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairPercentage(repairType, 50); + + + int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepair(repairType, 10); + + assertEquals(5, count); + } + + @Test + public void testDeleteAutoRepairHistory() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + AutoRepairUtils.deleteAutoRepairHistory(repairType, hostId); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(0, result.size()); + } + + @Test + public void testUpdateStartAutoRepairHistory() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + AutoRepairUtils.updateStartAutoRepairHistory(repairType, hostId, 123, AutoRepairUtils.RepairTurn.MY_TURN); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT repair_start_ts, repair_turn FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + UntypedResultSet.Row row = result.one(); + assertEquals(123, row.getLong(COL_REPAIR_START_TS, 0)); + assertEquals(AutoRepairUtils.RepairTurn.MY_TURN.toString(), row.getString(COL_REPAIR_TURN)); + } + + @Test + public void testUpdateFinishAutoRepairHistory() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + AutoRepairUtils.updateFinishAutoRepairHistory(repairType, hostId, 123); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT repair_finish_ts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + assertEquals(123, result.one().getLong(COL_REPAIR_FINISH_TS, 0)); + } + + @Test + public void testAddHostIdToDeleteHosts() + { + UUID otherHostId = UUID.randomUUID(); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), otherHostId)); + + AutoRepairUtils.addHostIdToDeleteHosts(repairType, hostId, otherHostId); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), otherHostId)); + assertNotNull(result); + assertEquals(1, result.size()); + Set deleteHosts = result.one().getSet(COL_DELETE_HOSTS, UUIDType.instance); + assertNotNull(deleteHosts); + assertEquals(1, deleteHosts.size()); + assertTrue(deleteHosts.contains(hostId)); + } + + @Test + public void testAddPriorityHost() + { + AutoRepairUtils.addPriorityHosts(repairType, ImmutableSet.of(localEndpoint)); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT * FROM %s.%s WHERE repair_type = '%s'", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString())); + assertNotNull(result); + assertEquals(1, result.size()); + Set repairPriority = result.one().getSet(COL_REPAIR_PRIORITY, UUIDType.instance); + assertNotNull(repairPriority); + assertEquals(1, repairPriority.size()); + assertTrue(repairPriority.contains(hostId)); + } + + @Test + public void testRemovePriorityStatus() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString(), hostId)); + + AutoRepairUtils.removePriorityStatus(repairType, hostId); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT * FROM %s.%s WHERE repair_type = '%s'", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString())); + assertNotNull(result); + assertEquals(1, result.size()); + Set repairPriority = result.one().getSet(COL_REPAIR_PRIORITY, UUIDType.instance); + assertNull(repairPriority); + } + + @Test + public void testGetPriorityHosts() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString(), hostId)); + + Set hosts = AutoRepairUtils.getPriorityHosts(repairType); + + assertNotNull(hosts); + assertEquals(1, hosts.size()); + assertTrue(hosts.contains(localEndpoint)); + } + + @Test + public void testCheckNodeContainsKeyspaceReplica() + { + Keyspace ks = Keyspace.open("ks"); + + assertTrue(AutoRepairUtils.shouldConsiderKeyspace(ks)); + } + + @Test + public void testTableMaxRepairTimeExceeded() + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairTableMaxRepairTime(repairType, "0s"); + + assertTrue(AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, 0)); + } + + @Test + public void testKeyspaceMaxRepairTimeExceeded() + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairTableMaxRepairTime(repairType, "0s"); + + assertTrue(AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, 0, 1)); + } + + @Test + public void testGetLastRepairFinishTime() + { + AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); + + assertEquals(0, history.getLastRepairFinishTime()); + + history.lastRepairFinishTime = 100; + + assertEquals(100, history.getLastRepairFinishTime()); + } + + @Test + public void testMyTurnToRunRepairShouldReturnMyTurnWhenRepairOngoing() + { + UUID myID = UUID.randomUUID(); + UUID otherID = UUID.randomUUID(); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 5); + long currentMillis = System.currentTimeMillis(); + // finish time less than start time means that repair is ongoing + AutoRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); + // finish time is larger than start time means that repair for other node is finished + AutoRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); + + assertEquals(AutoRepairUtils.RepairTurn.MY_TURN, AutoRepairUtils.myTurnToRunRepair(repairType, myID)); + } + + @Test + public void testLocalStrategyAndNetworkKeyspace() + { + assertFalse(AutoRepairUtils.shouldConsiderKeyspace(Keyspace.open("system"))); + assertTrue(AutoRepairUtils.shouldConsiderKeyspace(Keyspace.open(KEYSPACE))); + } + + @Test + public void testGetLastRepairTimeForNode() + { + UUID myID = UUID.randomUUID(); + UUID otherID = UUID.randomUUID(); + long currentMillis = System.currentTimeMillis(); + AutoRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); + AutoRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); + + assertEquals(currentMillis - 100, AutoRepairUtils.getLastRepairTimeForNode(repairType, myID)); + } + + @Test + public void testGetLastRepairTimeForNodeWhenHistoryIsEmpty() + { + UUID myID = UUID.randomUUID(); + + assertEquals(0, AutoRepairUtils.getLastRepairTimeForNode(repairType, myID)); + } + + @Test + public void testSkipSystemTraces() + { + assertFalse(AutoRepairUtils.shouldConsiderKeyspace(Keyspace.open(SchemaConstants.TRACE_KEYSPACE_NAME))); + } + + @Test + public void testHasNodesBelowMinimumVersionWithAllNodesAtMinVersion() throws UnknownHostException + { + Gossiper.instance.start(0); + Gossiper.instance.expireUpgradeFromVersion(); + + VersionedValue.VersionedValueFactory factory = new VersionedValue.VersionedValueFactory(null); + + // Set local endpoint to minimum version + EndpointState localEs = Gossiper.instance.getEndpointStateForEndpoint(localEndpoint); + if (localEs != null) + { + localEs.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("5.0.7")); + } + + // Add another node at minimum version + EndpointState es = new EndpointState((HeartBeatState) null); + es.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("5.0.7")); + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.100"), es); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.100")); + + boolean result = AutoRepairUtils.hasNodesBelowMinimumVersion(); + assertFalse(result); + } + + @Test + public void testHasNodesBelowMinimumVersionWithAllNodesAboveMinVersion() throws UnknownHostException + { + Gossiper.instance.start(0); + Gossiper.instance.expireUpgradeFromVersion(); + + VersionedValue.VersionedValueFactory factory = new VersionedValue.VersionedValueFactory(null); + + // Set local endpoint above minimum version + EndpointState localEs = Gossiper.instance.getEndpointStateForEndpoint(localEndpoint); + if (localEs != null) + { + localEs.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("5.0.8")); + } + + // Add another node above minimum version + EndpointState es = new EndpointState((HeartBeatState) null); + es.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("5.1.0")); + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.100"), es); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.100")); + + boolean result = AutoRepairUtils.hasNodesBelowMinimumVersion(); + assertFalse(result); + } + + @Test + public void testHasNodesBelowMinimumVersionWithOneNodeBelowMinVersion() throws UnknownHostException + { + Gossiper.instance.start(0); + Gossiper.instance.expireUpgradeFromVersion(); + + VersionedValue.VersionedValueFactory factory = new VersionedValue.VersionedValueFactory(null); + + // Set local endpoint at minimum version + EndpointState localEs = Gossiper.instance.getEndpointStateForEndpoint(localEndpoint); + if (localEs != null) + { + localEs.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("5.0.7")); + } + + // Add a node below minimum version + EndpointState es = new EndpointState((HeartBeatState) null); + es.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("5.0.6")); + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.100"), es); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.100")); + + boolean result = AutoRepairUtils.hasNodesBelowMinimumVersion(); + assertTrue(result); + } + + @Test + public void testHasNodesBelowMinimumVersionWithNullVersion() throws UnknownHostException + { + Gossiper.instance.start(0); + Gossiper.instance.expireUpgradeFromVersion(); + + VersionedValue.VersionedValueFactory factory = new VersionedValue.VersionedValueFactory(null); + + // Set local endpoint at minimum version + EndpointState localEs = Gossiper.instance.getEndpointStateForEndpoint(localEndpoint); + if (localEs != null) + { + localEs.addApplicationState(ApplicationState.RELEASE_VERSION, factory.releaseVersion("5.0.7")); + } + + // Add a node with no version set (simulates null version) + EndpointState es = new EndpointState((HeartBeatState) null); + // Note: Not setting RELEASE_VERSION to simulate null version + Gossiper.instance.endpointStateMap.put(InetAddressAndPort.getByName("127.0.0.100"), es); + Gossiper.instance.liveEndpoints.add(InetAddressAndPort.getByName("127.0.0.100")); + + boolean result = AutoRepairUtils.hasNodesBelowMinimumVersion(); + assertTrue(result); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitterHelper.java b/test/unit/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitterHelper.java new file mode 100644 index 000000000000..68edb3aae3b0 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitterHelper.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.dht.BootStrapper; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.locator.TokenMetadata; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.Pair; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.split; +import static org.apache.cassandra.repair.autorepair.FixedSplitTokenRangeSplitter.DEFAULT_NUMBER_OF_SUBRANGES; +import static org.apache.cassandra.repair.autorepair.FixedSplitTokenRangeSplitter.NUMBER_OF_SUBRANGES; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Helper class for {@link FixedSplitTokenRangeSplitterNoVNodesTest} and {@link FixedSplitTokenRangeSplitterVNodesTest} + */ +public class FixedSplitTokenRangeSplitterHelper extends CQLTester +{ + private static final String TABLE1 = "tbl1"; + private static final String TABLE2 = "tbl2"; + private static final String TABLE3 = "tbl3"; + public static final String KEYSPACE = "ks"; + public static final List tables = Arrays.asList(TABLE1, TABLE2, TABLE3); + public static final Map, AutoRepairUtils.SizeEstimate>> ksTablesEstimatedBytes = new HashMap<>(); + + public static void setupClass(int numTokens) throws Exception + { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + setAutoRepairEnabled(true); + + requireNetwork(); + AutoRepairUtils.setup(); + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1)); + + TokenMetadata metadata = StorageService.instance.getTokenMetadata(); + metadata.updateNormalTokens(BootStrapper.getRandomTokens(metadata, numTokens), FBUtilities.getBroadcastAddressAndPort()); + + Pair>, Integer> tokensAndWrappedAroundCount = getTokenRangesAndTotalWrapAroundCount(); + int totalToken = numTokens + tokensAndWrappedAroundCount.right(); + long perTokenSizeTable1 = 512L / totalToken; + long perTokenSizeTable2 = 1024L / totalToken; + long perTokenSizeTable3 = 2048L / totalToken; + for (Range tokenRange : tokensAndWrappedAroundCount.left) + { + ksTablesEstimatedBytes.put(AutoRepairUtils.getKeyspaceTableName(KEYSPACE, TABLE1), new HashMap, AutoRepairUtils.SizeEstimate>() + {{ + put(tokenRange, new AutoRepairUtils.SizeEstimate(AutoRepairConfig.RepairType.FULL, "", "", tokenRange, 0, perTokenSizeTable1, perTokenSizeTable1)); + }}); + ksTablesEstimatedBytes.put(AutoRepairUtils.getKeyspaceTableName(KEYSPACE, TABLE2), new HashMap, AutoRepairUtils.SizeEstimate>() + {{ + put(tokenRange, new AutoRepairUtils.SizeEstimate(AutoRepairConfig.RepairType.FULL, "", "", tokenRange, 0, perTokenSizeTable2, perTokenSizeTable2)); + }}); + ksTablesEstimatedBytes.put(AutoRepairUtils.getKeyspaceTableName(KEYSPACE, TABLE3), new HashMap, AutoRepairUtils.SizeEstimate>() + {{ + put(tokenRange, new AutoRepairUtils.SizeEstimate(AutoRepairConfig.RepairType.FULL, "", "", tokenRange, 0, perTokenSizeTable3, perTokenSizeTable3)); + }}); + } + } + + public static void testTokenRangesSplitByTable(int numTokens, int numberOfSubRanges, AutoRepairConfig.RepairType repairType) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairByKeyspace(repairType, false); + Pair>, Integer> tokensAndWrappedAroundCount = getTokenRangesAndTotalWrapAroundCount(); + Collection> tokens = tokensAndWrappedAroundCount.left(); + // For the test case, the tokens are allocated dynamically, so we do not know which token-ranges wrap around. + // As a result, we need to adjust the token count on a need basis. + numTokens += tokensAndWrappedAroundCount.right(); + assertEquals(numTokens, tokens.size()); + List> expectedToken = new ArrayList<>(); + int numberOfSplits = Math.max(1, numberOfSubRanges / tokens.size()); + for (int i = 0; i < tables.size(); i++) + { + for (Range token : tokens) + { + expectedToken.addAll(split(token, numberOfSplits)); + } + } + + Iterator keyspaceAssignments = + new FixedSplitTokenRangeSplitter(repairType, Collections.singletonMap(NUMBER_OF_SUBRANGES, Integer.toString(numberOfSubRanges))) + .getRepairAssignments(config.getRepairPrimaryTokenRangeOnly(repairType), getPlan(repairType)); + + // should be only 1 entry for the keyspace. + assertTrue(keyspaceAssignments.hasNext()); + KeyspaceRepairAssignments keyspaceRepairAssignment = keyspaceAssignments.next(); + assertFalse(keyspaceAssignments.hasNext()); + + List assignments = keyspaceRepairAssignment.getRepairAssignments(); + assertEquals(numTokens * numberOfSplits * tables.size(), assignments.size()); + assertEquals(expectedToken.size(), assignments.size()); + + int assignmentsPerTable = numTokens * numberOfSplits; + for (int i = 0; i < tables.size(); i++) + { + List assignmentForATable = new ArrayList<>(); + List> expectedTokensForATable = new ArrayList<>(); + for (int j = 0; j < assignmentsPerTable; j++) + { + long expectedBytes = ksTablesEstimatedBytes.get(AutoRepairUtils.getKeyspaceTableName(KEYSPACE, tables.get(i))).values().stream().mapToLong(sizeEstimate -> sizeEstimate.sizeForRepair).sum() / numberOfSplits; + int theTableAssignmentIdx = i * assignmentsPerTable + j; + assertEquals(expectedBytes, assignments.get(theTableAssignmentIdx).estimatedBytes); + assertEquals(Collections.singletonList(tables.get(i)), assignments.get(theTableAssignmentIdx).getTableNames()); + assignmentForATable.add(assignments.get(theTableAssignmentIdx)); + expectedTokensForATable.add(expectedToken.get(theTableAssignmentIdx)); + } + compare(numTokens, numberOfSplits, expectedTokensForATable, assignmentForATable); + } + } + + public static void testTokenRangesSplitByKeyspace(int numTokens, int numberOfSubRanges, AutoRepairConfig.RepairType repairType) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairByKeyspace(repairType, true); + Pair>, Integer> tokensAndWrappedRanges = getTokenRangesAndTotalWrapAroundCount(); + Collection> tokens = tokensAndWrappedRanges.left(); + // For the test case, the tokens are allocated dynamically, so we do not know which token-ranges wrap around. + // As a result, we need to adjust the token count on a need basis. + numTokens += tokensAndWrappedRanges.right(); + assertEquals(numTokens, tokens.size()); + int numberOfSplits = Math.max(1, numberOfSubRanges / tokens.size()); + List> expectedToken = new ArrayList<>(); + for (Range range : tokens) + { + expectedToken.addAll(AutoRepairUtils.split(range, numberOfSplits)); + } + + Iterator keyspaceAssignments = + new FixedSplitTokenRangeSplitter(repairType, Collections.singletonMap(NUMBER_OF_SUBRANGES, Integer.toString(numberOfSubRanges))) + .getRepairAssignments(config.getRepairPrimaryTokenRangeOnly(repairType), getPlan(repairType)); + + // should be only 1 entry for the keyspace. + assertTrue(keyspaceAssignments.hasNext()); + KeyspaceRepairAssignments keyspace = keyspaceAssignments.next(); + assertFalse(keyspaceAssignments.hasNext()); + + List assignments = keyspace.getRepairAssignments(); + assertNotNull(assignments); + + assertEquals(numTokens * numberOfSplits, assignments.size()); + assertEquals(expectedToken.size(), assignments.size()); + + compare(numTokens, numberOfSplits, expectedToken, assignments); + + for (int i = 0; i < assignments.size(); i++) + { + assertEquals(assignments.get(i).estimatedBytes, + ksTablesEstimatedBytes.values().stream() + .flatMap(tableMap -> tableMap.values().stream()) + .mapToLong(sizeEstimate -> sizeEstimate.sizeForRepair) + .sum() / numberOfSplits); + } + } + + public static void testTokenRangesWithDefaultSplit(int numTokens, AutoRepairConfig.RepairType repairType) + { + testTokenRangesSplitByKeyspace(numTokens, DEFAULT_NUMBER_OF_SUBRANGES, repairType); + } + + private static void compare(int numTokens, int numberOfSplits, List> expectedToken, List assignments) + { + assertEquals(expectedToken.size(), assignments.size()); + Set> a = new TreeSet<>(); + Set> b = new TreeSet<>(); + for (int i = 0; i < numTokens * numberOfSplits; i++) + { + a.add(expectedToken.get(i)); + b.add(assignments.get(i).getTokenRange()); + } + assertEquals(a, b); + } + + private static Pair>, Integer> getTokenRangesAndTotalWrapAroundCount() + { + int wrappedRanges = 0; + Collection> ranges = StorageService.instance.getPrimaryRangesForEndpoint(KEYSPACE, FBUtilities.getBroadcastAddressAndPort()); + Collection> tokens = new ArrayList<>(); + for (Range wrappedRange : ranges) + { + if (wrappedRange.isWrapAround()) + { + wrappedRanges++; + } + tokens.addAll(wrappedRange.unwrap()); + } + return Pair.create(tokens, wrappedRanges); + } + + private static List getPlan(AutoRepairConfig.RepairType repairType) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + List plan = PrioritizedRepairPlan.build(new HashMap>() + {{ + put(KEYSPACE, tables); + }}, repairType, (l) -> { + }, + config.getRepairPrimaryTokenRangeOnly(repairType)); + assertEquals(1, plan.size()); + assertEquals(1, plan.get(0).getKeyspaceRepairPlans().size()); + plan.get(0).getKeyspaceRepairPlans().get(0).ksTablesEstimatedBytes = ksTablesEstimatedBytes; + return plan; + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitterNoVNodesTest.java b/test/unit/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitterNoVNodesTest.java new file mode 100644 index 000000000000..a173299f4a73 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitterNoVNodesTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.cql3.CQLTester; + +/** + * Unit tests for a setup that does not have v-nodes {@link FixedSplitTokenRangeSplitter} + */ +@RunWith(Parameterized.class) +public class FixedSplitTokenRangeSplitterNoVNodesTest extends CQLTester +{ + private static final int numTokens = 1; + + @Parameterized.Parameter(0) + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameter(1) + public int numberOfSubRanges; + + @Parameterized.Parameters(name = "repairType={0}, numberOfSubRanges={1}") + public static Collection parameters() + { + List params = new ArrayList<>(); + for (AutoRepairConfig.RepairType type : AutoRepairConfig.RepairType.values()) + { + for (int subRange : Arrays.asList(1, 2, 4, 8, 16, 32, 64, 128, 256)) + { + params.add(new Object[]{ type, subRange }); + } + } + return params; + } + + @BeforeClass + public static void setupClass() throws Exception + { + FixedSplitTokenRangeSplitterHelper.setupClass(numTokens); + } + + @Test + public void testTokenRangesSplitByTable() + { + FixedSplitTokenRangeSplitterHelper.testTokenRangesSplitByTable(numTokens, numberOfSubRanges, repairType); + } + + @Test + public void testTokenRangesSplitByKeyspace() + { + FixedSplitTokenRangeSplitterHelper.testTokenRangesSplitByKeyspace(numTokens, numberOfSubRanges, repairType); + } + + @Test + public void testTokenRangesWithDefaultSplit() + { + FixedSplitTokenRangeSplitterHelper.testTokenRangesWithDefaultSplit(numTokens, repairType); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitterVNodesTest.java b/test/unit/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitterVNodesTest.java new file mode 100644 index 000000000000..d7ef654a3b21 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/FixedSplitTokenRangeSplitterVNodesTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.cql3.CQLTester; + +/** + * Unit tests for a setup that has v-nodes {@link FixedSplitTokenRangeSplitter} + */ +@RunWith(Parameterized.class) +public class FixedSplitTokenRangeSplitterVNodesTest extends CQLTester +{ + private static final int numTokens = 16; + + @Parameterized.Parameter(0) + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameter(1) + public int numberOfSubRanges; + + @Parameterized.Parameters(name = "repairType={0}, numberOfSubRanges={1}") + public static Collection parameters() + { + List params = new ArrayList<>(); + for (AutoRepairConfig.RepairType type : AutoRepairConfig.RepairType.values()) + { + for (int subRange : Arrays.asList(1, 2, 4, 8, 16, 32, 64, 128, 256)) + { + params.add(new Object[]{ type, subRange }); + } + } + return params; + } + + @BeforeClass + public static void setupClass() throws Exception + { + FixedSplitTokenRangeSplitterHelper.setupClass(numTokens); + } + + @Test + public void testTokenRangesSplitByTable() + { + FixedSplitTokenRangeSplitterHelper.testTokenRangesSplitByTable(numTokens, numberOfSubRanges, repairType); + } + + @Test + public void testTokenRangesSplitByKeyspace() + { + FixedSplitTokenRangeSplitterHelper.testTokenRangesSplitByKeyspace(numTokens, numberOfSubRanges, repairType); + } + + @Test + public void testTokenRangesWithDefaultSplit() + { + FixedSplitTokenRangeSplitterHelper.testTokenRangesWithDefaultSplit(numTokens, repairType); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java b/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java new file mode 100644 index 000000000000..d8eb428c4bda --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import com.google.common.collect.Lists; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.StorageService; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Unit tests for {@link org.apache.cassandra.repair.autorepair.PrioritizedRepairPlan} + */ +public class PrioritizedRepairPlanTest extends CQLTester +{ + + @BeforeClass + public static void setup() + { + StorageService.instance.doAutoRepairSetup(); + } + + @Test + public void testBuildWithDifferentPriorities() + { + // Test reordering assignments with different priorities + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '3'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '1'}"); + + List prioritizedRepairPlans = PrioritizedRepairPlan.build(new HashMap>(){{put(KEYSPACE, Arrays.asList(table1, table2, table3));}}, AutoRepairConfig.RepairType.FULL, (l) -> {}, + AutoRepairService.instance.getAutoRepairConfig().getRepairPrimaryTokenRangeOnly(AutoRepairConfig.RepairType.FULL)); + assertEquals(3, prioritizedRepairPlans.size()); + + // Verify the order is by descending priority and matches the expected tables + assertEquals(3, prioritizedRepairPlans.get(0).getPriority()); + assertEquals(table2, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + + assertEquals(2, prioritizedRepairPlans.get(1).getPriority()); + assertEquals(table1, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + + assertEquals(1, prioritizedRepairPlans.get(2).getPriority()); + assertEquals(table3, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + } + + @Test + public void testBuildWithSamePriority() + { + // Test reordering assignments with the same priority + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + + // Expect only 1 plan since all tables share the same priority + List prioritizedRepairPlans = PrioritizedRepairPlan.build(new HashMap>(){{put(KEYSPACE, Arrays.asList(table1, table2, table3));}}, AutoRepairConfig.RepairType.FULL, (l) -> {}, + AutoRepairService.instance.getAutoRepairConfig().getRepairPrimaryTokenRangeOnly(AutoRepairConfig.RepairType.FULL)); + assertEquals(1, prioritizedRepairPlans.size()); + + // Verify all tables present in the plan + assertEquals(1, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().size()); + KeyspaceRepairPlan keyspaceRepairPlan = prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0); + + List tableNames = keyspaceRepairPlan.getTableNames(); + assertEquals(3, tableNames.size()); + assertEquals(table1, tableNames.get(0)); + assertEquals(table2, tableNames.get(1)); + assertEquals(table3, tableNames.get(2)); + } + + @Test + public void testBuildWithMixedPriorities() + { + String ks1 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); + String table1 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table2 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '3'}"); + String table3 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table4 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '1'}"); + // No priority table should be bucketed at priority 0 + String table5 = createTable(ks1,"CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); + + // Create a new keyspace to ensure its tables get grouped with appropriate priority bucket + String ks2 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); + String table6 = createTable(ks2,"CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); + String table7 = createTable(ks2,"CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '1'}"); + + Map> keyspaceToTableMap = new HashMap<>(); + keyspaceToTableMap.put(ks1, Lists.newArrayList(table1, table2, table3, table4, table5)); + keyspaceToTableMap.put(ks2, Lists.newArrayList(table6, table7)); + + // Expect 4 plans + List prioritizedRepairPlans = PrioritizedRepairPlan.build(keyspaceToTableMap, AutoRepairConfig.RepairType.FULL, java.util.Collections::sort, true); + assertEquals(4, prioritizedRepairPlans.size()); + + // Verify the order is by descending priority and matches the expected tables + assertEquals(3, prioritizedRepairPlans.get(0).getPriority()); + assertEquals(1, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().size()); + assertEquals(ks1, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0).getKeyspaceName()); + assertEquals(table2, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + + assertEquals(2, prioritizedRepairPlans.get(1).getPriority()); + assertEquals(1, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().size()); + + assertEquals(ks1, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getKeyspaceName()); + assertEquals(table1, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + assertEquals(table3, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getTableNames().get(1)); + + assertEquals(1, prioritizedRepairPlans.get(2).getPriority()); + // 2 keyspaces should be present at priority 1 + assertEquals(2, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().size()); + // ks1.table4 expected in first plan + assertEquals(ks1, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(0).getKeyspaceName()); + assertEquals(table4, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + // ks2.table7 expected in second plan + assertEquals(ks2, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(1).getKeyspaceName()); + assertEquals(table7, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(1).getTableNames().get(0)); + + // Tables without priority should get bucketed at priority 0 + assertEquals(0, prioritizedRepairPlans.get(3).getPriority()); + // 2 keyspaces expected + assertEquals(2, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().size()); + // ks1.table5 expected in first plan + assertEquals(ks1, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().get(0).getKeyspaceName()); + assertEquals(table5, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + // ks2.table6 expected in second plan + assertEquals(ks2, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().get(1).getKeyspaceName()); + assertEquals(table6, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().get(1).getTableNames().get(0)); + } + + @Test + public void testBuildWithEmptyTableList() + { + // Test with an empty table list (should remain empty) + List prioritizedRepairPlans = PrioritizedRepairPlan.build(new HashMap>(){{put(KEYSPACE, Arrays.asList());}}, AutoRepairConfig.RepairType.FULL, (l) -> {}, + AutoRepairService.instance.getAutoRepairConfig().getRepairPrimaryTokenRangeOnly(AutoRepairConfig.RepairType.FULL)); + assertTrue(prioritizedRepairPlans.isEmpty()); + } + + @Test + public void testBuildWithOneTable() + { + // Test with a single element (should remain unchanged) + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '5'}"); + + // Expect only 1 plans + List prioritizedRepairPlans = PrioritizedRepairPlan.build(new HashMap>(){{put(KEYSPACE, Arrays.asList(table1));}}, AutoRepairConfig.RepairType.FULL, (l) -> {}, + AutoRepairService.instance.getAutoRepairConfig().getRepairPrimaryTokenRangeOnly(AutoRepairConfig.RepairType.FULL)); + assertEquals(1, prioritizedRepairPlans.size()); + + // Verify the order is by descending priority and matches the expected tables + assertEquals(5, prioritizedRepairPlans.get(0).getPriority()); + assertEquals(table1, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairTokenRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairTokenRangeSplitterTest.java new file mode 100644 index 000000000000..3161a4210c63 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairTokenRangeSplitterTest.java @@ -0,0 +1,592 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.auth.AuthKeyspace; +import org.apache.cassandra.config.DataStorageSpec.LongMebibytesBound; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter.FilteredRepairAssignments; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.SizeEstimate; +import org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter.SizedRepairAssignment; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SystemDistributedKeyspace; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.utils.concurrent.Refs; + +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.getKeyspaceTableName; +import static org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter.MAX_BYTES_PER_SCHEDULE; +import static org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter.BYTES_PER_ASSIGNMENT; +import static org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter.MAX_TABLES_PER_ASSIGNMENT; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Unit tests for {@link org.apache.cassandra.repair.autorepair.RepairTokenRangeSplitter} + */ +public class RepairTokenRangeSplitterTest extends CQLTester +{ + public static List SYSTEM_DISTRIBUTED_TABLE_NAMES = Arrays.asList(SystemDistributedKeyspace.REPAIR_HISTORY, SystemDistributedKeyspace.PARENT_REPAIR_HISTORY, SystemDistributedKeyspace.VIEW_BUILD_STATUS, SystemDistributedKeyspace.PARTITION_DENYLIST_TABLE, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY); + public static List AUTH_TABLE_NAMES = Arrays.asList(AuthKeyspace.ROLES, AuthKeyspace.ROLE_MEMBERS, AuthKeyspace.ROLE_PERMISSIONS, AuthKeyspace.RESOURCE_ROLE_INDEX, AuthKeyspace.NETWORK_PERMISSIONS, AuthKeyspace.CIDR_PERMISSIONS, AuthKeyspace.CIDR_GROUPS, AuthKeyspace.IDENTITY_TO_ROLES); + + private RepairTokenRangeSplitter repairRangeSplitter; + private String tableName; + private static Range FULL_RANGE; + + + @BeforeClass + public static void setUpClass() + { + CQLTester.setUpClass(); + AutoRepairService.setup(); + FULL_RANGE = new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken()); + requireNetwork(); + } + + @Before + public void setUp() + { + AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(RepairType.FULL, true); + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.FULL, Collections.emptyMap()); + tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); + } + + @Test + public void testSizePartitionCount() throws Throwable + { + insertAndFlushTable(tableName, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + try (Refs sstables = RepairTokenRangeSplitter.getSSTableReaderRefs(RepairType.FULL, KEYSPACE, tableName, FULL_RANGE)) + { + assertEquals(10, sstables.iterator().next().getEstimatedPartitionSize().count()); + SizeEstimate sizes = AutoRepairUtils.getSizesForRangeOfSSTables(RepairType.FULL, KEYSPACE, tableName, FULL_RANGE, sstables); + assertEquals(10, sizes.partitions); + } + } + + @Test + public void testSizePartitionCountSplit() throws Throwable + { + int partitionCount = 100_000; + int[] values = new int[partitionCount]; + for (int i = 0; i < values.length; i++) + values[i] = i + 1; + insertAndFlushTable(tableName, values); + Iterator> range = AutoRepairUtils.split(FULL_RANGE, 2).iterator(); + Range tokenRange1 = range.next(); + Range tokenRange2 = range.next(); + Assert.assertFalse(range.hasNext()); + + try (Refs sstables1 = RepairTokenRangeSplitter.getSSTableReaderRefs(RepairType.FULL, KEYSPACE, tableName, tokenRange1); + Refs sstables2 = RepairTokenRangeSplitter.getSSTableReaderRefs(RepairType.FULL, KEYSPACE, tableName, tokenRange2)) + { + SizeEstimate sizes1 = AutoRepairUtils.getSizesForRangeOfSSTables(RepairType.FULL, KEYSPACE, tableName, tokenRange1, sstables1); + SizeEstimate sizes2 = AutoRepairUtils.getSizesForRangeOfSSTables(RepairType.FULL, KEYSPACE, tableName, tokenRange2, sstables2); + + // +-5% because including entire compression blocks covering token range, HLL merge and the applying of range size approx ratio causes estimation errors + long allowableDelta = (long) (partitionCount * .05); + long estimatedPartitionDelta = Math.abs(partitionCount - (sizes1.partitions + sizes2.partitions)); + assertTrue("Partition count delta was +/-" + estimatedPartitionDelta + " but expected +/- " + allowableDelta, estimatedPartitionDelta <= allowableDelta); + } + } + + @Test + public void testGetRepairAssignmentsForTableNoSSTables() + { + // Should return 1 assignment if there are no SSTables + List assignments = repairRangeSplitter.getRepairAssignmentsForTable(new KeyspaceRepairPlan(CQLTester.KEYSPACE, Collections.singletonList(tableName), AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, Collections.singletonList(tableName), Collections.singletonList(FULL_RANGE))), tableName, FULL_RANGE); + assertEquals(1, assignments.size()); + } + + @Test + public void testGetRepairAssignmentsForTableSingle() throws Throwable + { + insertAndFlushSingleTable(); + List assignments = repairRangeSplitter.getRepairAssignmentsForTable(new KeyspaceRepairPlan(CQLTester.KEYSPACE, Collections.singletonList(tableName), AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, Collections.singletonList(tableName), Collections.singletonList(FULL_RANGE))), tableName, FULL_RANGE); + assertEquals(1, assignments.size()); + } + + @Test + public void testGetRepairAssignmentsForTableBatchingTablesCompressed() throws Throwable + { + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.FULL, Collections.singletonMap(MAX_TABLES_PER_ASSIGNMENT, "2")); + + List tableNames = createAndInsertTables(3, true); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, new KeyspaceRepairPlan(CQLTester.KEYSPACE, tableNames, AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, tableNames, Collections.singletonList(FULL_RANGE))), FULL_RANGE); + + // We expect two assignments, one with table1 and table2 batched, and one with table3 + assertEquals(2, assignments.size()); + assertEquals(2, assignments.get(0).getTableNames().size()); + assertEquals(1, assignments.get(1).getTableNames().size()); + assertEquals(new HashSet<>(Arrays.asList(tableNames.get(0), tableNames.get(1))), new HashSet<>(assignments.get(0).getTableNames())); + assertEquals(tableNames.get(2), assignments.get(1).getTableNames().get(0)); + } + + @Test + public void testGetRepairAssignmentsForTableBatchingTablesUncompressed() throws Throwable + { + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.FULL, Collections.singletonMap(MAX_TABLES_PER_ASSIGNMENT, "2")); + + List tableNames = createAndInsertTables(3, false); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, new KeyspaceRepairPlan(CQLTester.KEYSPACE, tableNames, AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, tableNames, Collections.singletonList(FULL_RANGE))), FULL_RANGE); + + // We expect two assignments, one with table1 and table2 batched, and one with table3 + assertEquals(2, assignments.size()); + assertEquals(2, assignments.get(0).getTableNames().size()); + assertEquals(1, assignments.get(1).getTableNames().size()); + assertEquals(new HashSet<>(Arrays.asList(tableNames.get(0), tableNames.get(1))), new HashSet<>(assignments.get(0).getTableNames())); + assertEquals(tableNames.get(2), assignments.get(1).getTableNames().get(0)); + assertTrue(assignments.get(0).getEstimatedBytes() > 0); + assertTrue(assignments.get(1).getEstimatedBytes() > 0); + assertEquals(calculateSSTableSizeOnDisk(Arrays.asList(tableNames.get(0), tableNames.get(1))), assignments.get(0).getEstimatedBytes()); + assertEquals(calculateSSTableSizeOnDisk(Collections.singletonList(tableNames.get(2))), assignments.get(1).getEstimatedBytes()); + } + + @Test + public void testGetRepairAssignmentsForTableBatchSizeCompressed() throws Throwable + { + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.FULL, Collections.singletonMap(MAX_TABLES_PER_ASSIGNMENT, "2")); + + List tableNames = createAndInsertTables(2, true); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, new KeyspaceRepairPlan(CQLTester.KEYSPACE, tableNames, AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, tableNames, Collections.singletonList(FULL_RANGE))), FULL_RANGE); + + // We expect one assignment, with two tables batched + assertEquals(1, assignments.size()); + assertEquals(2, assignments.get(0).getTableNames().size()); + assertEquals(new HashSet<>(Arrays.asList(tableNames.get(0), tableNames.get(1))), new HashSet<>(assignments.get(0).getTableNames())); + } + + @Test + public void testGetRepairAssignmentsForTableBatchSizeUnCompressed() throws Throwable + { + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.FULL, Collections.singletonMap(MAX_TABLES_PER_ASSIGNMENT, "2")); + + List tableNames = createAndInsertTables(2, false); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, + new KeyspaceRepairPlan(CQLTester.KEYSPACE, tableNames, AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, tableNames, Collections.singletonList(FULL_RANGE))), FULL_RANGE); + + // We expect one assignment, with two tables batched + assertEquals(1, assignments.size()); + assertEquals(2, assignments.get(0).getTableNames().size()); + assertEquals(new HashSet<>(Arrays.asList(tableNames.get(0), tableNames.get(1))), new HashSet<>(assignments.get(0).getTableNames())); + assertTrue(assignments.get(0).getEstimatedBytes() > 0); + assertEquals(calculateSSTableSizeOnDisk(Arrays.asList(tableNames.get(0), tableNames.get(1))), assignments.get(0).getEstimatedBytes()); + } + + @Test + public void testGetRepairAssignmentsForTableNoBatchingCompressed() throws Throwable + { + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.FULL, Collections.singletonMap(MAX_TABLES_PER_ASSIGNMENT, "1")); + + List tableNames = createAndInsertTables(3, true); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, new KeyspaceRepairPlan(CQLTester.KEYSPACE, tableNames, AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, tableNames, Collections.singletonList(FULL_RANGE))), FULL_RANGE); + + assertEquals(3, assignments.size()); + assertEquals(Collections.singletonList(tableNames.get(0)), assignments.get(0).getTableNames()); + assertEquals(Collections.singletonList(tableNames.get(1)), assignments.get(1).getTableNames()); + assertEquals(Collections.singletonList(tableNames.get(2)), assignments.get(2).getTableNames()); + } + + @Test + public void testGetRepairAssignmentsForTableNoBatchingUncompressed() throws Throwable + { + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.FULL, Collections.singletonMap(MAX_TABLES_PER_ASSIGNMENT, "1")); + + List tableNames = createAndInsertTables(3, false); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, new KeyspaceRepairPlan(CQLTester.KEYSPACE, tableNames, AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, tableNames, Collections.singletonList(FULL_RANGE))), FULL_RANGE); + + assertEquals(3, assignments.size()); + assertEquals(Collections.singletonList(tableNames.get(0)), assignments.get(0).getTableNames()); + assertEquals(Collections.singletonList(tableNames.get(1)), assignments.get(1).getTableNames()); + assertEquals(Collections.singletonList(tableNames.get(2)), assignments.get(2).getTableNames()); + assertTrue(assignments.get(0).getEstimatedBytes() > 0); + assertTrue(assignments.get(1).getEstimatedBytes() > 0); + assertTrue(assignments.get(2).getEstimatedBytes() > 0); + assertEquals(calculateSSTableSizeOnDisk(Collections.singletonList(tableNames.get(0))), assignments.get(0).getEstimatedBytes()); + assertEquals(calculateSSTableSizeOnDisk(Collections.singletonList(tableNames.get(1))), assignments.get(1).getEstimatedBytes()); + assertEquals(calculateSSTableSizeOnDisk(Collections.singletonList(tableNames.get(2))), assignments.get(2).getEstimatedBytes()); + } + + @Test + public void testGetRepairAssignmentsForTableAllBatchedCompressed() throws Throwable + { + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.FULL, Collections.singletonMap(MAX_TABLES_PER_ASSIGNMENT, "100")); + + List tableNames = createAndInsertTables(5, true); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, new KeyspaceRepairPlan(CQLTester.KEYSPACE, tableNames, AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, tableNames, Collections.singletonList(FULL_RANGE))), FULL_RANGE); + + assertEquals(1, assignments.size()); + assertEquals(5, assignments.get(0).getTableNames().size()); + assertEquals(new HashSet<>(tableNames), + new HashSet<>(assignments.get(0).getTableNames())); + } + + @Test + public void testGetRepairAssignmentsForTableAllBatchedUncompressed() throws Throwable + { + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.FULL, Collections.singletonMap(MAX_TABLES_PER_ASSIGNMENT, "100")); + + List tableNames = createAndInsertTables(5, false); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, new KeyspaceRepairPlan(CQLTester.KEYSPACE, tableNames, AutoRepairUtils.calcTotalBytesToBeRepaired(RepairType.FULL, CQLTester.KEYSPACE, tableNames, Collections.singletonList(FULL_RANGE))), FULL_RANGE); + + assertEquals(1, assignments.size()); + assertEquals(5, assignments.get(0).getTableNames().size()); + assertEquals(new HashSet<>(tableNames), + new HashSet<>(assignments.get(0).getTableNames())); + assertTrue(assignments.get(0).getEstimatedBytes() > 0); + assertEquals(calculateSSTableSizeOnDisk(tableNames), assignments.get(0).getEstimatedBytes()); + } + + @Test(expected = IllegalStateException.class) + public void testMergeEmptyAssignments() + { + // Test when the list of assignments is empty + List emptyAssignments = Collections.emptyList(); + RepairTokenRangeSplitter.merge(emptyAssignments); + } + + @Test + public void testMergeSingleAssignment() + { + // Test when there is only one assignment in the list + String keyspaceName = "testKeyspace"; + List tableNames = Arrays.asList("table1", "table2"); + + SizedRepairAssignment assignment = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames); + List assignments = Collections.singletonList(assignment); + + SizedRepairAssignment result = RepairTokenRangeSplitter.merge(assignments); + + assertEquals(FULL_RANGE, result.getTokenRange()); + assertEquals(keyspaceName, result.getKeyspaceName()); + assertEquals(new HashSet<>(tableNames), new HashSet<>(result.getTableNames())); + } + + @Test + public void testMergeMultipleAssignmentsWithSameTokenRangeAndKeyspace() + { + // Test merging multiple assignments with the same token range and keyspace + String keyspaceName = "testKeyspace"; + List tableNames1 = Arrays.asList("table1", "table2"); + List tableNames2 = Arrays.asList("table2", "table3"); + + SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames1); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames2); + List assignments = Arrays.asList(assignment1, assignment2); + + SizedRepairAssignment result = RepairTokenRangeSplitter.merge(assignments); + + assertEquals(FULL_RANGE, result.getTokenRange()); + assertEquals(keyspaceName, result.getKeyspaceName()); + assertEquals(new HashSet<>(Arrays.asList("table1", "table2", "table3")), new HashSet<>(result.getTableNames())); + } + + @Test(expected = IllegalStateException.class) + public void testMergeDifferentTokenRange() + { + // Test merging assignments with different token ranges + Iterator> range = AutoRepairUtils.split(FULL_RANGE, 2).iterator(); // Split the full range into two ranges ie (0-100, 100-200 + Range tokenRange1 = range.next(); + Range tokenRange2 = range.next(); + Assert.assertFalse(range.hasNext()); + + String keyspaceName = "testKeyspace"; + List tableNames = Arrays.asList("table1", "table2"); + + SizedRepairAssignment assignment1 = new SizedRepairAssignment(tokenRange1, keyspaceName, tableNames); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(tokenRange2, keyspaceName, tableNames); + List assignments = Arrays.asList(assignment1, assignment2); + + RepairTokenRangeSplitter.merge(assignments); // Should throw IllegalStateException + } + + @Test(expected = IllegalStateException.class) + public void testMergeDifferentKeyspaceName() + { + // Test merging assignments with different keyspace names + List tableNames = Arrays.asList("table1", "table2"); + + SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, "keyspace1", tableNames); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, "keyspace2", tableNames); + List assignments = Arrays.asList(assignment1, assignment2); + + RepairTokenRangeSplitter.merge(assignments); // Should throw IllegalStateException + } + + @Test + public void testMergeWithDuplicateTables() + { + // Test merging assignments with duplicate table names + String keyspaceName = "testKeyspace"; + List tableNames1 = Arrays.asList("table1", "table2"); + List tableNames2 = Arrays.asList("table2", "table3"); + + SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames1); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames2); + List assignments = Arrays.asList(assignment1, assignment2); + + RepairAssignment result = RepairTokenRangeSplitter.merge(assignments); + + // The merged result should contain all unique table names + assertEquals(new HashSet<>(Arrays.asList("table1", "table2", "table3")), new HashSet<>(result.getTableNames())); + } + + @Test + public void testGetRepairAssignmentsSplitsBySubrangeSizeAndFilterLimitsByMaxBytesPerSchedule() + { + // Ensures that getRepairAssignments splits by BYTES_PER_ASSIGNMENT and filterRepairAssignments limits by MAX_BYTES_PER_SCHEDULE. + repairRangeSplitter = new RepairTokenRangeSplitter(RepairType.INCREMENTAL, Collections.emptyMap()); + repairRangeSplitter.setParameter(BYTES_PER_ASSIGNMENT, "50GiB"); + repairRangeSplitter.setParameter(MAX_BYTES_PER_SCHEDULE, "100GiB"); + + // Given a size estimate of 1024GiB, we should expect 21 splits (50GiB*21 = 1050GiB < 1024GiB) + SizeEstimate sizeEstimate = sizeEstimateByBytes(new LongMebibytesBound("1024GiB")); + + List assignments = repairRangeSplitter.getRepairAssignments(sizeEstimate); + + // Should be 21 assignments, each being ~48.76 GiB + assertEquals(21, assignments.size()); + long expectedBytes = 52357696560L; + for (int i = 0; i < assignments.size(); i++) + { + SizedRepairAssignment assignment = assignments.get(i); + assertEquals("Did not get expected value for assignment " + i, 52357696560L, assignment.getEstimatedBytes()); + } + + // When filtering we should only get 2 assignments back (48.76 * 2 < 100GiB) + FilteredRepairAssignments filteredRepairAssignments = repairRangeSplitter.filterRepairAssignments(0, KEYSPACE, assignments, 0); + List finalRepairAssignments = filteredRepairAssignments.repairAssignments; + assertEquals(2, finalRepairAssignments.size()); + assertEquals(expectedBytes * 2, filteredRepairAssignments.newBytesSoFar); + } + + @Test + public void testTokenRangesRepairByKeyspace() + { + AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(RepairType.FULL, true); + + Map, AutoRepairUtils.SizeEstimate>> ksTablesEstimatedBytes = new HashMap<>(); + List> tokenRanges = AutoRepairUtils.getTokenRanges(true, SchemaConstants.AUTH_KEYSPACE_NAME); + long tableSizeInBytes = 100L; + long tableSizeInBytesPerTokenRange = tableSizeInBytes / tokenRanges.size(); + for (String tableName : AUTH_TABLE_NAMES) + { + String ksTableName = getKeyspaceTableName(SchemaConstants.AUTH_KEYSPACE_NAME, tableName); + ksTablesEstimatedBytes.putIfAbsent(ksTableName, new HashMap<>()); + Map, AutoRepairUtils.SizeEstimate> rangeSizeEstimateMap = ksTablesEstimatedBytes.get(ksTableName); + + for (Range tokenRange : tokenRanges) + { + rangeSizeEstimateMap.put(tokenRange, new AutoRepairUtils.SizeEstimate(AutoRepairConfig.RepairType.FULL, SchemaConstants.AUTH_KEYSPACE_NAME, tableName, tokenRange, 0, tableSizeInBytesPerTokenRange, tableSizeInBytesPerTokenRange)); + } + } + final KeyspaceRepairPlan repairPlan = new KeyspaceRepairPlan(SchemaConstants.AUTH_KEYSPACE_NAME, new ArrayList<>(AUTH_TABLE_NAMES), ksTablesEstimatedBytes); + assertEquals(tableSizeInBytes * AUTH_TABLE_NAMES.size(), repairPlan.getEstimatedBytes()); + + final PrioritizedRepairPlan prioritizedRepairPlan = new PrioritizedRepairPlan(0, Arrays.asList(repairPlan)); + + Iterator keyspaceAssignments = repairRangeSplitter.getRepairAssignments(true, Arrays.asList(prioritizedRepairPlan)); + + // should be only 1 entry for the keyspace. + assertTrue(keyspaceAssignments.hasNext()); + KeyspaceRepairAssignments keyspace = keyspaceAssignments.next(); + assertFalse(keyspaceAssignments.hasNext()); + + List assignments = keyspace.getRepairAssignments(); + assertNotNull(assignments); + + // Should only be two assignments (since single node encompasses the whole range, should get 2 primary ranges) + // to account for the range wrapping the ring. + assertEquals(2, assignments.size()); + + for (RepairAssignment assignment : assignments) + { + assertEquals(AUTH_TABLE_NAMES.size(), assignment.getTableNames().size()); + } + } + + @Test + public void testTokenRangesRepairByKeyspaceFalse() + { + AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(RepairType.FULL, false); + + Map, AutoRepairUtils.SizeEstimate>> ksTablesEstimatedBytes = new HashMap<>(); + List> tokenRanges = AutoRepairUtils.getTokenRanges(true, SchemaConstants.AUTH_KEYSPACE_NAME); + long tableSizeInBytes = 100L; + long tableSizeInBytesPerTokenRange = tableSizeInBytes / tokenRanges.size(); + for (String tableName : AUTH_TABLE_NAMES) + { + String ksTableName = getKeyspaceTableName(SchemaConstants.AUTH_KEYSPACE_NAME, tableName); + ksTablesEstimatedBytes.putIfAbsent(ksTableName, new HashMap<>()); + Map, AutoRepairUtils.SizeEstimate> rangeSizeEstimateMap = ksTablesEstimatedBytes.get(ksTableName); + + for (Range tokenRange : tokenRanges) + { + rangeSizeEstimateMap.put(tokenRange, new AutoRepairUtils.SizeEstimate(AutoRepairConfig.RepairType.FULL, SchemaConstants.AUTH_KEYSPACE_NAME, tableName, tokenRange, 0, tableSizeInBytesPerTokenRange, tableSizeInBytesPerTokenRange)); + } + } + + final KeyspaceRepairPlan repairPlan = new KeyspaceRepairPlan(SchemaConstants.AUTH_KEYSPACE_NAME, new ArrayList<>(AUTH_TABLE_NAMES), ksTablesEstimatedBytes); + assertEquals(tableSizeInBytes * AUTH_TABLE_NAMES.size(), repairPlan.getEstimatedBytes()); + + final PrioritizedRepairPlan prioritizedRepairPlan = new PrioritizedRepairPlan(0, Arrays.asList(repairPlan)); + + Iterator keyspaceAssignments = repairRangeSplitter.getRepairAssignments(true, Arrays.asList(prioritizedRepairPlan)); + + // should be only 1 entry for the keyspace. + assertTrue(keyspaceAssignments.hasNext()); + KeyspaceRepairAssignments keyspace = keyspaceAssignments.next(); + assertFalse(keyspaceAssignments.hasNext()); + + List assignments = keyspace.getRepairAssignments(); + assertNotNull(assignments); + + // Should be two ranges * X system_auth table names assignments + assertEquals(2 * AUTH_TABLE_NAMES.size(), assignments.size()); + + // each assignment should only include one table. + for (RepairAssignment assignment : assignments) + { + assertEquals(1, assignment.getTableNames().size()); + } + } + + @Test(expected = IllegalArgumentException.class) + public void testSetParameterShouldNotAllowUnknownParameter() + { + repairRangeSplitter.setParameter("unknown", "x"); + } + + @Test(expected = IllegalArgumentException.class) + public void testSetParameterShouldNotAllowSettingBytesPerAssignmentGreaterThanMaxBytesPerSchedule() + { + repairRangeSplitter.setParameter(MAX_BYTES_PER_SCHEDULE, "500GiB"); + repairRangeSplitter.setParameter(BYTES_PER_ASSIGNMENT, "600GiB"); + } + + @Test(expected = IllegalArgumentException.class) + public void testSetParameterShouldNotAllowSettingMaxBytesPerScheduleLessThanBytesPerAssignment() + { + repairRangeSplitter.setParameter(BYTES_PER_ASSIGNMENT, "100MiB"); + repairRangeSplitter.setParameter(MAX_BYTES_PER_SCHEDULE, "50MiB"); + } + + @Test + public void testGetParameters() + { + repairRangeSplitter.setParameter(BYTES_PER_ASSIGNMENT, "100MiB"); + repairRangeSplitter.setParameter(MAX_TABLES_PER_ASSIGNMENT, "5"); + + Map parameters = repairRangeSplitter.getParameters(); + // Each parameter should be present. + assertEquals(RepairTokenRangeSplitter.PARAMETERS.size(), parameters.size()); + // The parameters we explicitly set should be set exactly as we set them. + assertEquals("100MiB", parameters.get(BYTES_PER_ASSIGNMENT)); + assertEquals("5", parameters.get(MAX_TABLES_PER_ASSIGNMENT)); + } + + private SizeEstimate sizeEstimateByBytes(LongMebibytesBound totalSize) + { + return sizeEstimateByBytes(totalSize, totalSize); + } + + private SizeEstimate sizeEstimateByBytes(LongMebibytesBound sizeInRange, LongMebibytesBound totalSize) + { + return new SizeEstimate(RepairType.INCREMENTAL, KEYSPACE, "table1", FULL_RANGE, 1, sizeInRange.toBytes(), totalSize.toBytes()); + } + + private void insertAndFlushSingleTable() throws Throwable + { + execute("INSERT INTO %s (k, v) values (?, ?)", 1, 1); + flush(); + } + + private List createAndInsertTables(int count, boolean enableCompression) throws Throwable + { + List tableNames = new ArrayList<>(); + for (int i = 0; i < count; i++) + { + String tableName; + if (enableCompression) + { + tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); + } + else + { + tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH compression = { 'enabled' : false }"); + } + tableNames.add(tableName); + insertAndFlushTable(tableName); + } + return tableNames; + } + + private long calculateSSTableSizeOnDisk(List tableNames) + { + long totalSSTableBytes = 0; + for (int i = 0; i < tableNames.size(); i++) + { + String tableName = tableNames.get(i); + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, tableName); + assertNotNull(cfs); + Iterable sstables = cfs.getTracker().getView().select(SSTableSet.CANONICAL); + for (SSTableReader sstable : sstables) + { + totalSSTableBytes += sstable.onDiskLength(); + } + } + return totalSSTableBytes; + } + + private void insertAndFlushTable(String tableName) throws Throwable + { + insertAndFlushTable(tableName, 1); + } + + private void insertAndFlushTable(String tableName, int... vals) throws Throwable + { + for (int i : vals) + { + executeFormattedQuery("INSERT INTO " + KEYSPACE + '.' + tableName + " (k, v) values (?, ?)", i, i); + } + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, tableName); + cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java b/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java new file mode 100644 index 000000000000..d9a7f0ee7af9 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.net.UnknownHostException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.service.StorageService; + +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + + +/** + * Unit tests to cover AutoRepair functionality inside {@link org.apache.cassandra.service.StorageService} + */ +public class SSTableRepairedAtTest extends CQLTester +{ + public static final String TEST_KEYSPACE = "test_keyspace"; + public static ColumnFamilyStore table1; + public static ColumnFamilyStore table2; + + @BeforeClass + public static void setUp() throws ConfigurationException, UnknownHostException + { + requireNetwork(); + AutoRepairUtils.setup(); + StorageService.instance.doAutoRepairSetup(); + DatabaseDescriptor.setCDCEnabled(false); + } + + @Before + public void clearData() + { + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + QueryProcessor.executeInternal(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", TEST_KEYSPACE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (key text, val text, primary key(key))", TEST_KEYSPACE, "table1")); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (key text, val text, primary key(key))", TEST_KEYSPACE, "table2")); + + Keyspace.open(TEST_KEYSPACE).getColumnFamilyStore("table1").truncateBlocking(); + Keyspace.open(TEST_KEYSPACE).getColumnFamilyStore("table2").truncateBlocking(); + + table1 = Keyspace.open(TEST_KEYSPACE).getColumnFamilyStore("table1"); + assert table1 != null; + table2 = Keyspace.open(TEST_KEYSPACE).getColumnFamilyStore("table2"); + assert table2 != null; + } + + @Test + public void testGetTablesForKeyspace() + { + List result = StorageService.instance.getTablesForKeyspace(TEST_KEYSPACE); + + assertEquals(Arrays.asList(table1.name, table2.name), result.stream().sorted().collect(Collectors.toList())); + } + + @Test + public void testGetTablesForKeyspaceNotFound() + { + String missingKeyspace = "MISSING_KEYSPACE"; + try + { + StorageService.instance.getTablesForKeyspace(missingKeyspace); + fail("Expected an AssertionError to be thrown"); + } + catch (AssertionError e) + { + assertEquals("Unknown keyspace " + missingKeyspace, e.getMessage()); + } + } + + @Test + public void testMutateSSTableRepairedStateTableNotFound() + { + try + { + StorageService.instance.mutateSSTableRepairedState(true, false, TEST_KEYSPACE, List.of("MISSING_TABLE")); + fail("Expected an InvalidRequestException to be thrown"); + } + catch (RuntimeException e) + { + assertEquals("Table MISSING_TABLE does not exist in keyspace " + TEST_KEYSPACE, e.getMessage()); + // Test passed + } + } + + @Test + public void testMutateSSTableRepairedStateTablePreview() + { + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + assertEquals(1, table1.getLiveSSTables().size()); + + List result = StorageService.instance.mutateSSTableRepairedState(true, true, TEST_KEYSPACE, Arrays.asList(table1.name)); + + assertEquals(1, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); + }); + } + + @Test + public void testMutateSSTableRepairedStateTableRepaired() + { + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + assertEquals(2, table1.getLiveSSTables().size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + }); + + List result = StorageService.instance.mutateSSTableRepairedState(true, false, TEST_KEYSPACE, Arrays.asList(table1.name)); + + assertEquals(2, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertTrue(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); + }); + } + + @Test + public void testMutateSSTableRepairedStateTableUnrepaired() throws Exception + { + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + table1.getCompactionStrategyManager().mutateRepaired(table1.getLiveSSTables(), 1, null, false); + assertEquals(2, table1.getLiveSSTables().stream().filter(SSTableReader::isRepaired).count()); + + List result = StorageService.instance.mutateSSTableRepairedState(false, false, TEST_KEYSPACE, Arrays.asList(table1.name)); + + assertEquals(2, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); + }); + } +} diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java index 3b7d0cd9a095..8f5b64710fa7 100644 --- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java +++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java @@ -1,21 +1,21 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyright ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, -* software distributed under the License is distributed on an -* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -* KIND, either express or implied. See the License for the -* specific language governing permissions and limitations -* under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.cassandra.service; import java.util.ArrayList; @@ -36,8 +36,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; +import org.apache.cassandra.service.disk.usage.DiskUsageMonitor; import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.concurrent.Condition; + +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -66,6 +69,7 @@ import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.concurrent.Refs; +import org.mockito.Mock; import static org.apache.cassandra.repair.messages.RepairOption.DATACENTERS_KEY; import static org.apache.cassandra.repair.messages.RepairOption.FORCE_REPAIR_KEY; @@ -79,6 +83,8 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; public class ActiveRepairServiceTest { @@ -90,13 +96,18 @@ public class ActiveRepairServiceTest public String cfname; public ColumnFamilyStore store; public InetAddressAndPort LOCAL, REMOTE; - + @Mock + public DiskUsageMonitor diskUsageMonitor; + private static DiskUsageMonitor originalDiskUsageMonitor; + private static double originalRepairDiskHeadroomRejectRatio; private boolean initialized; @BeforeClass public static void defineSchema() throws ConfigurationException { SchemaLoader.prepareServer(); + originalDiskUsageMonitor = DiskUsageMonitor.instance; + originalRepairDiskHeadroomRejectRatio = DatabaseDescriptor.getRepairDiskHeadroomRejectRatio(); SchemaLoader.createKeyspace(KEYSPACE5, KeyspaceParams.simple(2), SchemaLoader.standardCFMD(KEYSPACE5, CF_COUNTER), @@ -121,6 +132,14 @@ public void prepare() throws Exception StorageService.instance.setTokens(Collections.singleton(tmd.partitioner.getRandomToken())); tmd.updateNormalToken(tmd.partitioner.getMinimumToken(), REMOTE); assert tmd.isMember(REMOTE); + initMocks(this); + } + + @After + public void tearDown() + { + DiskUsageMonitor.instance = originalDiskUsageMonitor; + DatabaseDescriptor.setRepairDiskHeadroomRejectRatio(originalRepairDiskHeadroomRejectRatio); } @Test @@ -226,7 +245,7 @@ public void testGetNeighborsTimesTwoInSpecifiedHosts() throws Throwable } expected.remove(FBUtilities.getBroadcastAddressAndPort()); - Collection hosts = Arrays.asList(FBUtilities.getBroadcastAddressAndPort().getHostAddressAndPort(),expected.get(0).getHostAddressAndPort()); + Collection hosts = Arrays.asList(FBUtilities.getBroadcastAddressAndPort().getHostAddressAndPort(), expected.get(0).getHostAddressAndPort()); Iterable> ranges = StorageService.instance.getLocalReplicas(KEYSPACE5).ranges(); assertEquals(expected.get(0), ActiveRepairService.instance().getNeighbors(KEYSPACE5, ranges, @@ -244,7 +263,6 @@ public void testGetNeighborsSpecifiedHostsWithNoLocalHost() throws Throwable ActiveRepairService.instance().getNeighbors(KEYSPACE5, ranges, ranges.iterator().next(), null, hosts); } - @Test public void testParentRepairStatus() throws Throwable { @@ -262,7 +280,6 @@ public void testParentRepairStatus() throws Throwable List failed = StorageService.instance.getParentRepairStatus(3); assertNotNull(failed); assertEquals(ActiveRepairService.ParentRepairStatus.FAILED, ActiveRepairService.ParentRepairStatus.valueOf(failed.get(0))); - } Set addTokens(int max) throws Throwable @@ -334,10 +351,10 @@ private static RepairOption opts(String... params) { assert params.length % 2 == 0 : "unbalanced key value pairs"; Map opt = new HashMap<>(); - for (int i=0; i<(params.length >> 1); i++) + for (int i = 0; i < (params.length >> 1); i++) { int idx = i << 1; - opt.put(params[idx], params[idx+1]); + opt.put(params[idx], params[idx + 1]); } return RepairOption.parse(opt, DatabaseDescriptor.getPartitioner()); } @@ -415,7 +432,8 @@ public void testRejectWhenPoolFullStrategy() throws InterruptedException // Submission is unblocked Thread.sleep(250); - validationExecutor.submit(() -> {}); + validationExecutor.submit(() -> { + }); } finally { @@ -452,8 +470,8 @@ public void testQueueWhenPoolFullStrategy() throws InterruptedException allSubmitted.await(TASK_SECONDS + 1, TimeUnit.SECONDS); // Give the tasks we expect to execute immediately chance to be scheduled - Util.spinAssertEquals(2 , ((ExecutorPlus) validationExecutor)::getActiveTaskCount, 1); - Util.spinAssertEquals(3 , ((ExecutorPlus) validationExecutor)::getPendingTaskCount, 1); + Util.spinAssertEquals(2, ((ExecutorPlus) validationExecutor)::getActiveTaskCount, 1); + Util.spinAssertEquals(3, ((ExecutorPlus) validationExecutor)::getPendingTaskCount, 1); // verify that we've reached a steady state with 2 threads actively processing and 3 queued tasks Assert.assertEquals(2, ((ExecutorPlus) validationExecutor).getActiveTaskCount()); @@ -502,6 +520,41 @@ public void testRepairSessionSpaceInMiB() } } + @Test + public void testVerifyDefaultDiskHeadroomThreshold() + { + Assert.assertTrue(ActiveRepairService.instance().verifyDiskHeadroomThreshold(TimeUUID.maxAtUnixMillis(0), PreviewKind.NONE)); + } + + @Test + public void testVerifyDiskHeadroomThresholdDiskFull() + { + DiskUsageMonitor.instance = diskUsageMonitor; + when(diskUsageMonitor.getDiskUsage()).thenReturn(1.0); + DatabaseDescriptor.setRepairDiskHeadroomRejectRatio(1.0); + + Assert.assertFalse(ActiveRepairService.instance().verifyDiskHeadroomThreshold(TimeUUID.maxAtUnixMillis(0), PreviewKind.NONE)); + } + + @Test + public void testVerifyDiskHeadroomThresholdSufficientDisk() + { + DiskUsageMonitor.instance = diskUsageMonitor; + when(diskUsageMonitor.getDiskUsage()).thenReturn(0.0); + DatabaseDescriptor.setRepairDiskHeadroomRejectRatio(0.0); + + Assert.assertTrue(ActiveRepairService.instance().verifyDiskHeadroomThreshold(TimeUUID.maxAtUnixMillis(0), PreviewKind.NONE)); + } + + @Test(expected = RuntimeException.class) + public void testPrepareForRepairThrowsExceptionForInsufficientDisk() + { + DiskUsageMonitor.instance = diskUsageMonitor; + when(diskUsageMonitor.getDiskUsage()).thenReturn(1.5); + + ActiveRepairService.instance().prepareForRepair(TimeUUID.maxAtUnixMillis(0), null, null, opts(INCREMENTAL_KEY, b2s(true)), false, null); + } + private static class Task implements Runnable { private final Condition blocked; diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java new file mode 100644 index 000000000000..07b8bcc69ec3 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; + +import static org.junit.Assert.assertEquals; + +/** + * Unit tests for {@link org.apache.cassandra.service.AutoRepairService} + */ +public class AutoRepairServiceBasicTest extends CQLTester +{ + private static AutoRepairService autoRepairService; + private static AutoRepairConfig config; + + @Before + public void setUp() + { + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); + config = new AutoRepairConfig(); + autoRepairService = new AutoRepairService(); + autoRepairService.config = config; + } + + @Test + public void testSetup() + { + AutoRepairService.instance.config = null; + + AutoRepairService.setup(); + + assertEquals(DatabaseDescriptor.getAutoRepairConfig(), AutoRepairService.instance.config); + } + + @Test + public void testGetAutoRepairConfigReturnsConfig() + { + assertEquals(config, autoRepairService.getAutoRepairConfig()); + } + + @Test + public void testsetAutoRepairHistoryClearDeleteHostsBufferInSecV2() + { + autoRepairService.setAutoRepairHistoryClearDeleteHostsBufferDuration("100s"); + + assertEquals(100, config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds()); + } + + @Test + public void testsetAutoRepairMaxRetriesCount() + { + autoRepairService.setAutoRepairMaxRetriesCount(AutoRepairConfig.RepairType.INCREMENTAL.name(), 101); + + assertEquals(101, config.getRepairMaxRetries(AutoRepairConfig.RepairType.INCREMENTAL)); + } + + @Test + public void testsetAutoRepairRetryBackoffInSec() + { + autoRepairService.setAutoRepairRetryBackoff(AutoRepairConfig.RepairType.INCREMENTAL.name(), "102s"); + + assertEquals(102, config.getRepairRetryBackoff(AutoRepairConfig.RepairType.INCREMENTAL).toSeconds()); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsWithSchedulerDisabled() + { + autoRepairService.config = new AutoRepairConfig(false); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL.name(), true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayButMVRepairDisabled() + { + autoRepairService.config = new AutoRepairConfig(true); + autoRepairService.config.setMaterializedViewRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL.name(), true); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() + { + autoRepairService.config = new AutoRepairConfig(true); + autoRepairService.config.setMaterializedViewRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, true); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL.name(), true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() + { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setMaterializedViewsEnabled(true); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL.name(), true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithCDCReplayButCDCDisabled() + { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setCDCOnRepairEnabled(true); + DatabaseDescriptor.setCDCEnabled(false); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL.name(), true); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsForIRWithCDCReplay() + { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setCDCOnRepairEnabled(true); + DatabaseDescriptor.setCDCEnabled(true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL.name(), true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithCDCReplayDisabled() + { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setCDCEnabled(true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL.name(), true); + } +} diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java new file mode 100644 index 000000000000..9c2af3e1c793 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service; + +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Set; +import java.util.UUID; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.junit.Assert.assertEquals; + +/** + * Unit tests covering different repair types for {@link org.apache.cassandra.service.AutoRepairService} + */ +@RunWith(Parameterized.class) +public class AutoRepairServiceRepairTypeTest extends CQLTester +{ + @Parameterized.Parameter() + public AutoRepairConfig.RepairType repairType; + + private final UUID host1 = UUID.fromString("00000000-0000-0000-0000-000000000001"); + private final UUID host2 = UUID.fromString("00000000-0000-0000-0000-000000000002"); + + private AutoRepairService instance; + + @Parameterized.Parameters(name = "repairType={0}") + public static Collection repairTypes() + { + return Arrays.asList(AutoRepairConfig.RepairType.values()); + } + + @BeforeClass + public static void setupClass() throws Exception + { + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + setAutoRepairEnabled(true); + requireNetwork(); + } + + @Before + public void setUpTest() + { + AutoRepairUtils.setup(); + instance = new AutoRepairService(); + } + + @Test + public void testGetOnGoingRepairHostIdsTest() + { + long now = System.currentTimeMillis(); + AutoRepairUtils.insertNewRepairHistory(repairType, host1, now, now - 1000000); + AutoRepairUtils.insertNewRepairHistory(repairType, host2, now, now - 1000000); + + Set hosts = instance.getOnGoingRepairHostIds(repairType.name()); + + assertEquals(ImmutableSet.of(host1.toString(), host2.toString()), hosts); + } +} diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java new file mode 100644 index 000000000000..10a4fe30c1f5 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service; + +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SystemDistributedKeyspace; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Set; +import java.util.UUID; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Unit tests for (updating parameters through JMX) {@link org.apache.cassandra.service.AutoRepairService} + */ +@RunWith(Parameterized.class) +public class AutoRepairServiceSetterTest extends CQLTester +{ + private static final AutoRepairConfig config = new AutoRepairConfig(true); + + @Parameterized.Parameter + public AutoRepairConfig.RepairType repairTypeStr; + + @Parameterized.Parameter(1) + public T arg; + + @Parameterized.Parameter(2) + public BiConsumer setter; + + @Parameterized.Parameter(3) + public Function getter; + + @Parameterized.Parameters(name = "{index}: repairType={0}, arg={1}") + public static Collection testCases() + { + DatabaseDescriptor.daemonInitialization(); + return Stream.of( + forEachRepairType(true, AutoRepairService.instance::setAutoRepairEnabled, config::isAutoRepairEnabled), + forEachRepairType(100, AutoRepairService.instance::setRepairThreads, config::getRepairThreads), + forEachRepairType(400, AutoRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), + forEachRepairType(ImmutableSet.of("dc1", "dc2"), AutoRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), + forEachRepairType(true, AutoRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), + forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentage, config::getParallelRepairPercentage), + forEachRepairType(700, AutoRepairService.instance::setParallelRepairCount, config::getParallelRepairCount), + forEachRepairType(true, AutoRepairService.instance::setMVRepairEnabled, config::getMaterializedViewRepairEnabled), + forEachRepairType(InetAddressAndPort.getLocalHost().getHostAddressAndPort(), (repairType, commaSeparatedHostSet) -> AutoRepairService.instance.setRepairPriorityForHosts(repairType, (String) commaSeparatedHostSet), AutoRepairUtils::getPriorityHosts), + forEachRepairType(InetAddressAndPort.getLocalHost().getHostAddressAndPort(), (repairType, commaSeparatedHostSet) -> AutoRepairService.instance.setForceRepairForHosts(repairType, (String) commaSeparatedHostSet), AutoRepairServiceSetterTest::isLocalHostForceRepair) + ).flatMap(Function.identity()).collect(Collectors.toList()); + } + + private static Set isLocalHostForceRepair(AutoRepairConfig.RepairType type) + { + UUID hostId = StorageService.instance.getHostIdForEndpoint(InetAddressAndPort.getLocalHost()); + UntypedResultSet resultSet = QueryProcessor.executeInternal(String.format( + "SELECT force_repair FROM %s.%s WHERE host_id = %s and repair_type = '%s'", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, hostId, type)); + + if (!resultSet.isEmpty() && resultSet.one().getBoolean("force_repair")) + { + return ImmutableSet.of(InetAddressAndPort.getLocalHost()); + } + return ImmutableSet.of(); + } + + private static Stream forEachRepairType(T arg, BiConsumer setter, Function getter) + { + Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + testCases[repairType.ordinal()] = new Object[]{ repairType, arg, setter, getter }; + } + + return Arrays.stream(testCases); + } + + @BeforeClass + public static void setup() throws Exception + { + DatabaseDescriptor.daemonInitialization(); + setAutoRepairEnabled(true); + requireNetwork(); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); + AutoRepairUtils.setup(); + AutoRepairService.instance.config = config; + } + + @Before + public void prepare() + { + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); + } + + @Test + public void testSettersTest() + { + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + DatabaseDescriptor.setCDCOnRepairEnabled(false); + setter.accept(repairTypeStr.name(), arg); + T actualConfig = getter.apply(repairTypeStr); + if (actualConfig instanceof Set) + // When performing a setRepairPriorityForHosts or setForceRepairForHosts, a comma-separated list of + // ip addresses is provided as input. The configuration is expected to return a Set of Strings that + // represent the configured IP addresses. This especial handling allows verification of this special + // case where one of the entries in the Set must match the configured input. + assertThat(actualConfig).satisfiesAnyOf(entry -> assertThat(entry.toString()).contains(arg.toString())); + else + assertThat(actualConfig).isEqualTo(arg); + } +} diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index d4cf4504f8f8..c46af59bd766 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -1,21 +1,21 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyright ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, -* software distributed under the License is distributed on an -* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -* KIND, either express or implied. See the License for the -* specific language governing permissions and limitations -* under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.cassandra.service; @@ -23,12 +23,17 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.util.*; +import java.util.stream.Collectors; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.SystemKeyspace; + import org.junit.Assert; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -43,6 +48,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.distributed.shared.WithProperties; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.gms.ApplicationState; import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.gms.VersionedValue; @@ -59,10 +65,17 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIP_DISABLE_THREAD_VALIDATION; import static org.apache.cassandra.config.CassandraRelevantProperties.REPLACE_ADDRESS; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + public class StorageServiceServerTest { + public static final String keyspace = "test_keyspace"; + public static ColumnFamilyStore table1; + public static ColumnFamilyStore table2; + @BeforeClass public static void setUp() throws ConfigurationException { @@ -75,6 +88,21 @@ public static void setUp() throws ConfigurationException mkdirs(); cleanup(); StorageService.instance.initServer(0); + + SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), + SchemaLoader.standardCFMD(keyspace, "table1").build(), + SchemaLoader.standardCFMD(keyspace, "table2").build()); + table1 = Keyspace.open(keyspace).getColumnFamilyStore("table1"); + assert table1 != null; + table2 = Keyspace.open(keyspace).getColumnFamilyStore("table2"); + assert table2 != null; + } + + @Before + public void clearData() + { + table1.truncateBlocking(); + table2.truncateBlocking(); } @Test @@ -555,6 +583,7 @@ public void testCreateRepairRangeFrom() throws Exception /** * Test that StorageService.getNativeAddress returns the correct value based on available yaml and gossip state + * * @throws Exception */ @Test @@ -566,7 +595,7 @@ public void testGetNativeAddress() throws Exception //Default to using the provided address with the configured port assertEquals("127.0.0.2:" + DatabaseDescriptor.getNativeTransportPort(), StorageService.instance.getNativeaddress(internalAddress, true)); - VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(Murmur3Partitioner.instance); + VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(Murmur3Partitioner.instance); //If we don't have the port use the gossip address, but with the configured port Gossiper.instance.getEndpointStateForEndpoint(internalAddress).addApplicationState(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(InetAddress.getByName("127.0.0.3"))); assertEquals("127.0.0.3:" + DatabaseDescriptor.getNativeTransportPort(), StorageService.instance.getNativeaddress(internalAddress, true)); @@ -587,7 +616,7 @@ public void testGetNativeAddressIPV6() throws Exception //Default to using the provided address with the configured port assertEquals("[0:0:0:0:0:0:0:3]:" + DatabaseDescriptor.getNativeTransportPort(), StorageService.instance.getNativeaddress(internalAddressIPV6, true)); - VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(Murmur3Partitioner.instance); + VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(Murmur3Partitioner.instance); //If RPC_ADDRESS is present with an IPv6 address, we should properly bracket encode the IP with the configured port. Gossiper.instance.getEndpointStateForEndpoint(internalAddressIPV6).addApplicationState(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(InetAddress.getByName("0:0:0:0:0:0:5a:3"))); assertEquals("[0:0:0:0:0:0:5a:3]:" + DatabaseDescriptor.getNativeTransportPort(), StorageService.instance.getNativeaddress(internalAddressIPV6, true)); @@ -639,7 +668,7 @@ public void isReplacingSameHostAddressAndHostIdTest() throws UnknownHostExceptio try (WithProperties properties = new WithProperties()) { UUID differentHostId = UUID.randomUUID(); - Assert.assertFalse(StorageService.instance.isReplacingSameHostAddressAndHostId(differentHostId)); + assertFalse(StorageService.instance.isReplacingSameHostAddressAndHostId(differentHostId)); final String hostAddress = FBUtilities.getBroadcastAddressAndPort().getHostAddress(false); UUID localHostId = SystemKeyspace.getOrInitializeLocalHostId(); @@ -658,4 +687,98 @@ public void isReplacingSameHostAddressAndHostIdTest() throws UnknownHostExceptio Assert.assertFalse(StorageService.instance.isReplacingSameHostAddressAndHostId(differentHostId)); } } + + @Test + public void testGetTablesForKeyspace() + { + List result = StorageService.instance.getTablesForKeyspace(keyspace); + + assertEquals(Arrays.asList(table1.name, table2.name), result.stream().sorted().collect(Collectors.toList())); + } + + @Test + public void testGetTablesForKeyspaceNotFound() + { + String missingKeyspace = "MISSING_KEYSPACE"; + try + { + StorageService.instance.getTablesForKeyspace(missingKeyspace); + fail("Expected an AssertionError to be thrown"); + } + catch (AssertionError e) + { + assertEquals("Unknown keyspace " + missingKeyspace, e.getMessage()); + } + } + + @Test + public void testMutateSSTableRepairedStateTableNotFound() + { + try + { + StorageService.instance.mutateSSTableRepairedState(true, false, keyspace, Arrays.asList("MISSING_TABLE")); + fail("Expected an InvalidRequestException to be thrown"); + } + catch (RuntimeException e) + { + assertEquals("Table MISSING_TABLE does not exist in keyspace " + keyspace, e.getMessage()); + // Test passed + } + } + + @Test + public void testMutateSSTableRepairedStateTablePreview() + { + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + assertEquals(1, table1.getLiveSSTables().size()); + + List result = StorageService.instance.mutateSSTableRepairedState(true, true, keyspace, Arrays.asList(table1.name)); + + assertEquals(1, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); + }); + } + + @Test + public void testMutateSSTableRepairedStateTableRepaired() + { + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + assertEquals(2, table1.getLiveSSTables().size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + }); + + List result = StorageService.instance.mutateSSTableRepairedState(true, false, keyspace, Arrays.asList(table1.name)); + + assertEquals(2, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertTrue(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); + }); + } + + @Test + public void testMutateSSTableRepairedStateTableUnrepaired() throws Exception + { + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + table1.getCompactionStrategyManager().mutateRepaired(table1.getLiveSSTables(), 1, null, false); + assertEquals(2, table1.getLiveSSTables().stream().filter(SSTableReader::isRepaired).count()); + + List result = StorageService.instance.mutateSSTableRepairedState(false, false, keyspace, Arrays.asList(table1.name)); + + assertEquals(2, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); + }); + } } \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java b/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java index 87b9ff93fded..df0a2c1be2ea 100644 --- a/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java +++ b/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java @@ -50,6 +50,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.utils.BreaksJMX; import org.assertj.core.api.Assertions; import org.reflections.Reflections; @@ -98,6 +101,10 @@ public class JMXStandardsTest .add(IllegalStateException.class) .add(ClassNotFoundException.class) .add(OpenDataException.class) + .add(InvalidRequestException.class) + .add(AutoRepairConfig.RepairType.class) + .add(InetAddressAndPort.class) + .add(AutoRepairConfig.class) .build(); /** * This list is a set of types under java.* and javax.*, but are too vague that could cause issues; this does not diff --git a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java new file mode 100644 index 000000000000..82293581d807 --- /dev/null +++ b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.tools.nodetool; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.Arrays; +import java.util.Collection; + +import com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.Output; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link org.apache.cassandra.tools.nodetool.AutoRepairStatus} + */ +@RunWith(Parameterized.class) +public class AutoRepairStatusTest +{ + @Mock + private static NodeProbe probe; + + private ByteArrayOutputStream cmdOutput; + + private static AutoRepairStatus cmd; + + @Parameterized.Parameter() + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameters(name = "repairType={0}") + public static Collection repairTypes() + { + return Arrays.asList(AutoRepairConfig.RepairType.values()); + } + + @Before + public void setUp() throws Exception + { + MockitoAnnotations.initMocks(this); + cmdOutput = new ByteArrayOutputStream(); + PrintStream out = new PrintStream(cmdOutput); + when(probe.output()).thenReturn(new Output(out, out)); + cmd = new AutoRepairStatus(); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.loadConfig(); + setAutoRepairEnabled(true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.FULL, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, true); + } + + @Test(expected = IllegalArgumentException.class) + public void testExecuteWithoutRepairType() + { + cmd.repairType = null; + cmd.execute(probe); + } + + @Test + public void testExecuteWithNoNodes() + { + cmd.repairType = repairType.name(); + + cmd.execute(probe); + assertEquals("Active Repairs\n" + + "NONE \n", cmdOutput.toString()); + } + + @Test + public void testExecute() + { + when(probe.getAutoRepairOnGoingRepairHostIds(repairType.name())).thenReturn(ImmutableSet.of("host1", "host2", "host3", "host4")); + cmd.repairType = repairType.name(); + + cmd.execute(probe); + + assertEquals("Active Repairs \n" + + "host1,host2,host3,host4\n", cmdOutput.toString()); + } +} diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java new file mode 100644 index 000000000000..5d23d22253ad --- /dev/null +++ b/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.tools.nodetool; + +import java.io.OutputStream; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.Output; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link org.apache.cassandra.tools.nodetool.SSTableRepairedSetTest} + */ +public class SSTableRepairedSetTest +{ + @Mock + private NodeProbe probe; + + private SSTableRepairedSet cmd; + + @Before + public void setUp() + { + MockitoAnnotations.initMocks(this); + PrintStream noopStream = new PrintStream(new OutputStream() + { + @Override + public void write(int b) + { + } + }); + when(probe.output()).thenReturn(new Output(noopStream, noopStream)); + cmd = new SSTableRepairedSet(); + } + + @Test + public void testNoKeyspace() + { + when(probe.getNonLocalStrategyKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("ks1", "ks2"))); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("ks1", "ks2"))); + when(probe.getAutoRepairTablesForKeyspace("ks1")).thenReturn(new ArrayList<>(Arrays.asList("table1", "table2"))); + when(probe.getAutoRepairTablesForKeyspace("ks2")).thenReturn(new ArrayList<>(Arrays.asList("table3", "table4"))); + cmd.isRepaired = true; + cmd.reallySet = true; + + cmd.execute(probe); + + verify(probe, times(1)).mutateSSTableRepairedState(true, false, "ks1", Arrays.asList("table1", "table2")); + verify(probe, times(1)).mutateSSTableRepairedState(true, false, "ks2", Arrays.asList("table3", "table4")); + } + + @Test + public void testBothRepairedAndUnrepaired() + { + cmd.args = Arrays.asList("keyspace"); + cmd.isRepaired = true; + cmd.isUnrepaired = true; + cmd.execute(probe); + verify(probe, never()).mutateSSTableRepairedState(anyBoolean(), anyBoolean(), anyString(), anyList()); + } + + @Test + public void testNeitherRepairedNorUnrepaired() + { + cmd.args = Arrays.asList("keyspace"); + cmd.execute(probe); + verify(probe, never()).mutateSSTableRepairedState(anyBoolean(), anyBoolean(), anyString(), anyList()); + } + + @Test + public void testRepairedPreview() + { + cmd.args = Arrays.asList("keyspace"); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("keyspace"))); + cmd.isRepaired = true; + cmd.execute(probe); + verify(probe).mutateSSTableRepairedState(true, true, "keyspace", new ArrayList<>()); + } + + @Test + public void testUnrepairedReallySet() + { + cmd.args = Arrays.asList("keyspace"); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("keyspace"))); + cmd.isUnrepaired = true; + cmd.reallySet = true; + cmd.execute(probe); + verify(probe).mutateSSTableRepairedState(false, false, "keyspace", new ArrayList<>()); + } + + @Test + public void testExecuteWithTableNames() + { + cmd.args = Arrays.asList("keyspace", "table1", "table2"); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("keyspace"))); + cmd.isRepaired = true; + cmd.reallySet = true; + cmd.execute(probe); + verify(probe).mutateSSTableRepairedState(true, false, "keyspace", Arrays.asList("table1", "table2")); + } +} diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java new file mode 100644 index 000000000000..e57ec27acf34 --- /dev/null +++ b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java @@ -0,0 +1,320 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.tools.nodetool; + +import java.io.PrintStream; +import java.util.Arrays; +import java.util.Collection; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Suite; + +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.tools.NodeProbe; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.junit.Assert.fail; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** + * Unit tests for {@link org.apache.cassandra.tools.nodetool.SetAutoRepairConfig} + */ +@RunWith(Suite.class) +@Suite.SuiteClasses({ SetAutoRepairConfigTest.NoParamTests.class, SetAutoRepairConfigTest.RepairTypeParamTests.class, + SetAutoRepairConfigTest.RepairTypeAndArgsParamsTests.class }) +public class SetAutoRepairConfigTest +{ + protected static SetAutoRepairConfig cmd; + + public static void before(NodeProbe probeMock, PrintStream outMock) + { + when(probeMock.isAutoRepairDisabled()).thenReturn(false); + cmd = new SetAutoRepairConfig(); + cmd.out = outMock; + } + + public static class NoParamTests + { + @Mock + private static NodeProbe probe; + + @Mock + private static PrintStream out; + + @Before + public void setUp() + { + MockitoAnnotations.initMocks(this); + before(probe, out); + } + + @Test + public void testHistoryDeleteHostsClearBufferInSec() + { + cmd.args = ImmutableList.of("history_clear_delete_hosts_buffer_interval", "1s"); + + cmd.execute(probe); + + verify(probe, times(1)).setAutoRepairHistoryClearDeleteHostsBufferDuration("1s"); + + // test scenario when auto repair is disabled + when(probe.isAutoRepairDisabled()).thenReturn(true); + + cmd.execute(probe); + + // test new calls are not made when auto repair is disabled + verify(probe, times(1)).setAutoRepairHistoryClearDeleteHostsBufferDuration("1s"); + } + + + @Test + public void testMinRepairDuration() + { + cmd.args = ImmutableList.of("min_repair_task_duration", "4s"); + + cmd.execute(probe); + + verify(probe, times(1)).setAutoRepairMinRepairTaskDuration("4s"); + } + + @Test + public void testStartScheduler() + { + cmd.args = ImmutableList.of("start_scheduler", "false"); + + cmd.execute(probe); + + verify(probe, times(0)).startAutoRepairScheduler(); + + cmd.args = ImmutableList.of("start_scheduler", "true"); + + cmd.execute(probe); + + verify(probe, times(1)).startAutoRepairScheduler(); + } + } + + @RunWith(Parameterized.class) + public static class RepairTypeParamTests + { + @Mock + private static NodeProbe probe; + + @Mock + private static PrintStream out; + + @Parameterized.Parameter + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameters(name = "repairType={0}") + public static Object[] data() + { + return AutoRepairConfig.RepairType.values(); + } + + private static InetAddressAndPort localEndpoint; + private static InetAddressAndPort otherEndpoint; + + @Before + public void setUp() throws Exception + { + MockitoAnnotations.initMocks(this); + before(probe, out); + localEndpoint = InetAddressAndPort.getByName("127.0.0.1:7000"); + otherEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 1); + } + + @Test(expected = IllegalArgumentException.class) + public void testNoArgs() + { + cmd.repairTypeStr = repairType.name(); + cmd.execute(probe); + } + + @Test + public void testRepairSchedulingDisabled() + { + when(probe.isAutoRepairDisabled()).thenReturn(true); + cmd.repairTypeStr = repairType.name(); + cmd.args = ImmutableList.of("threads", "1"); + + cmd.execute(probe); + + verify(out, times(1)).println("Auto-repair is not enabled"); + verify(probe, times(0)).setAutoRepairThreads(repairType.name(), 1); + } + + @Test + public void testRepairTypeDisabled() + { + cmd.repairTypeStr = repairType.name(); + cmd.args = ImmutableList.of("number_of_repair_threads", "1"); + + cmd.execute(probe); + + verify(probe, times(1)).setAutoRepairThreads(repairType.name(), 1); + } + + @Test + public void testV2FlagMissing() + { + cmd.repairTypeStr = repairType.name(); + cmd.args = ImmutableList.of("threads", "1"); + + try + { + cmd.execute(probe); + + fail("expected IllegalArgumentException"); + } + catch (IllegalArgumentException e) + { + // expected + } + + verify(probe, times(0)).setAutoRepairThreads(repairType.name(), 0); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidParamType() + { + cmd.repairTypeStr = repairType.name(); + cmd.args = ImmutableList.of("unknown_type", "1"); + + cmd.execute(probe); + } + + @Test + public void testPriorityHosts() + { + String commaSeparatedHostSet = String.join(",", localEndpoint.toString().substring(1), otherEndpoint.toString().substring(1)); + cmd.repairTypeStr = repairType.name(); + cmd.args = ImmutableList.of("priority_hosts", commaSeparatedHostSet); + + cmd.execute(probe); + + verify(probe, times(1)).setAutoRepairPriorityForHosts(repairType.name(), commaSeparatedHostSet); + } + + @Test + public void testForceRepairHosts() + { + String commaSeparatedHostSet = String.join(",", localEndpoint.toString().substring(1), otherEndpoint.toString().substring(1)); + cmd.repairTypeStr = repairType.name(); + cmd.args = ImmutableList.of("forcerepair_hosts", commaSeparatedHostSet); + + cmd.execute(probe); + + verify(probe, times(1)).setAutoRepairForceRepairForHosts(repairType.name(), commaSeparatedHostSet); + } + } + + @RunWith(Parameterized.class) + public static class RepairTypeAndArgsParamsTests + { + @Parameterized.Parameter + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameter(1) + public String paramType; + + @Parameterized.Parameter(2) + public String paramVal; + + @Parameterized.Parameter(3) + public Consumer verifyFunc; + + @Parameterized.Parameters(name = "repairType={0},paramType={1}") + public static Collection testCases() + { + return Stream.of( + forEachRepairType("enabled", "true", (type) -> verify(probe, times(1)).setAutoRepairEnabled(type.name(), true)), + forEachRepairType("number_of_repair_threads", "1", (type) -> verify(probe, times(1)).setAutoRepairThreads(type.name(), 1)), + forEachRepairType("min_repair_interval", "3h", (type) -> verify(probe, times(1)).setAutoRepairMinInterval(type.name(), "3h")), + forEachRepairType("sstable_upper_threshold", "4", (type) -> verify(probe, times(1)).setAutoRepairSSTableCountHigherThreshold(type.name(), 4)), + forEachRepairType("table_max_repair_time", "5s", (type) -> verify(probe, times(1)).setAutoRepairTableMaxRepairTime(type.name(), "5s")), + forEachRepairType("repair_primary_token_range_only", "true", (type) -> verify(probe, times(1)).setAutoRepairPrimaryTokenRangeOnly(type.name(), true)), + forEachRepairType("parallel_repair_count", "6", (type) -> verify(probe, times(1)).setAutoRepairParallelRepairCount(type.name(), 6)), + forEachRepairType("parallel_repair_percentage", "7", (type) -> verify(probe, times(1)).setAutoRepairParallelRepairPercentage(type.name(), 7)), + forEachRepairType("materialized_view_repair_enabled", "true", (type) -> verify(probe, times(1)).setAutoRepairMaterializedViewRepairEnabled(type.name(), true)), + forEachRepairType("ignore_dcs", "dc1,dc2", (type) -> verify(probe, times(1)).setAutoRepairIgnoreDCs(type.name(), ImmutableSet.of("dc1", "dc2"))), + forEachRepairType("token_range_splitter.max_bytes_per_schedule", "500GiB", (type) -> verify(probe, times(1)).setAutoRepairTokenRangeSplitterParameter(type.name(), "max_bytes_per_schedule", "500GiB")), + forEachRepairType("repair_max_retries", "3", (type) -> verify(probe, times(1)).setAutoRepairMaxRetriesCount(type.name(), 3)), + forEachRepairType("repair_retry_backoff", "60s", (type) -> verify(probe, times(1)).setAutoRepairRetryBackoff(type.name(), "60s")), + forEachRepairType("mixed_major_version_repair_enabled", "false", (type) -> verify(probe, times(1)).setMixedMajorVersionRepairEnabled(false)) + ).flatMap(Function.identity()).collect(Collectors.toList()); + } + + private static Stream forEachRepairType(String paramType, String paramVal, Consumer verifyFunc) + { + Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + testCases[repairType.ordinal()] = new Object[]{ repairType, paramType, paramVal, verifyFunc }; + } + + return Arrays.stream(testCases); + } + + @Mock + private static NodeProbe probe; + + @Mock + private static PrintStream out; + + @Before + public void setUp() + { + MockitoAnnotations.initMocks(this); + before(probe, out); + } + + @Test + public void test() + { + cmd.repairTypeStr = repairType.name(); + cmd.args = ImmutableList.of(paramType, paramVal); + + cmd.execute(probe); + + verifyFunc.accept(repairType); + + // test scenario when auto repair is disabled + when(probe.isAutoRepairDisabled()).thenReturn(true); + + cmd.execute(probe); + + // test new calls are not made when auto repair is disabled + verifyFunc.accept(repairType); + } + } +} diff --git a/test/unit/org/apache/cassandra/tools/nodetool/TableHistogramsTest.java b/test/unit/org/apache/cassandra/tools/nodetool/TableHistogramsTest.java index 733b876c092c..d6cf5e715a0e 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/TableHistogramsTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/TableHistogramsTest.java @@ -43,7 +43,7 @@ public class TableHistogramsTest extends CQLTester SchemaKeyspace.metadata().tables.size() + TraceKeyspace.TABLE_NAMES.size() + AuthKeyspace.TABLE_NAMES.size() + - SystemDistributedKeyspace.TABLE_NAMES.size(); + SystemDistributedKeyspace.getTableNames().size(); @BeforeClass public static void setup() throws Exception