diff --git a/CHANGES.txt b/CHANGES.txt
index 7680b8d841cc..a90144186ae0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,25 @@
-4.0
+4.0-alpha2
+ * Extract an AbstractCompactionController to allow for custom implementations (CASSANDRA-15286)
+ * Move chronicle-core version from snapshot to stable, and include carrotsearch in generated pom.xml (CASSANDRA-15321)
+ * Untangle RepairMessage sub-hierarchy of messages, use new messaging (more) correctly (CASSANDRA-15163)
+ * Add `allocate_tokens_for_local_replication_factor` option for token allocation (CASSANDRA-15260)
+ * Add Alibaba Cloud Platform snitch (CASSANDRA-15092)
+
+
+4.0-alpha1
+ * Inaccurate exception message with nodetool snapshot (CASSANDRA-15287)
+ * Fix InternodeOutboundMetrics overloaded bytes/count mixup (CASSANDRA-15186)
+ * Enhance & reenable RepairTest with compression=off and compression=on (CASSANDRA-15272)
+ * Improve readability of Table metrics Virtual tables units (CASSANDRA-15194)
+ * Fix error with non-existent table for nodetool tablehistograms (CASSANDRA-14410)
+ * Avoid result truncation in decimal operations (CASSANDRA-15232)
+ * Catch non-IOException in FileUtils.close to make sure that all resources are closed (CASSANDRA-15225)
+ * Align load column in nodetool status output (CASSANDRA-14787)
+ * CassandraNetworkAuthorizer uses cached roles info (CASSANDRA-15089)
+ * Introduce optional timeouts for idle client sessions (CASSANDRA-11097)
+ * Fix AlterTableStatement dropped type validation order (CASSANDRA-15203)
+ * Update Netty dependencies to latest, clean up SocketFactory (CASSANDRA-15195)
+ * Native Transport - Apply noSpamLogger to ConnectionLimitHandler (CASSANDRA-15167)
* Reduce heap pressure during compactions (CASSANDRA-14654)
* Support building Cassandra with JDK 11 (CASSANDRA-15108)
* Use quilt to patch cassandra.in.sh in Debian packaging (CASSANDRA-14710)
@@ -358,13 +379,30 @@
* nodetool clearsnapshot requires --all to clear all snapshots (CASSANDRA-13391)
* Correctly count range tombstones in traces and tombstone thresholds (CASSANDRA-8527)
* cqlshrc.sample uses incorrect option for time formatting (CASSANDRA-14243)
+ * Multi-version in-JVM dtests (CASSANDRA-14937)
+ * Allow instance class loaders to be garbage collected for inJVM dtest (CASSANDRA-15170)
3.11.5
+ * Make sure user defined compaction transactions are always closed (CASSANDRA-15123)
* Fix cassandra-env.sh to use $CASSANDRA_CONF to find cassandra-jaas.config (CASSANDRA-14305)
* Fixed nodetool cfstats printing index name twice (CASSANDRA-14903)
* Add flag to disable SASI indexes, and warnings on creation (CASSANDRA-14866)
Merged from 3.0:
+ * Fix LegacyLayout RangeTombstoneList IndexOutOfBoundsException when upgrading and RangeTombstone bounds are asymmetric (CASSANDRA-15172)
+ * Fix NPE when using allocate_tokens_for_keyspace on new DC/rack (CASSANDRA-14952)
+ * Filter sstables earlier when running cleanup (CASSANDRA-15100)
+ * Use mean row count instead of mean column count for index selectivity calculation (CASSANDRA-15259)
+ * Avoid updating unchanged gossip states (CASSANDRA-15097)
+ * Prevent recreation of previously dropped columns with a different kind (CASSANDRA-14948)
+ * Prevent client requests from blocking on executor task queue (CASSANDRA-15013)
+ * Toughen up column drop/recreate type validations (CASSANDRA-15204)
+ * LegacyLayout should handle paging states that cross a collection column (CASSANDRA-15201)
+ * Prevent RuntimeException when username or password is empty/null (CASSANDRA-15198)
+ * Multiget thrift query returns null records after digest mismatch (CASSANDRA-14812)
+ * Handle paging states serialized with a different version than the session's (CASSANDRA-15176)
+ * Throw IOE instead of asserting on unsupporter peer versions (CASSANDRA-15066)
+ * Update token metadata when handling MOVING/REMOVING_TOKEN events (CASSANDRA-15120)
* Add ability to customize cassandra log directory using $CASSANDRA_LOG_DIR (CASSANDRA-15090)
* Fix assorted gossip races and add related runtime checks (CASSANDRA-15059)
* cassandra-stress works with frozen collections: list and set (CASSANDRA-14907)
@@ -375,6 +413,7 @@ Merged from 3.0:
* Add missing commands to nodetool_completion (CASSANDRA-14916)
* Anti-compaction temporarily corrupts sstable state for readers (CASSANDRA-15004)
Merged from 2.2:
+ * Handle exceptions during authentication/authorization (CASSANDRA-15041)
* Support cross version messaging in in-jvm upgrade dtests (CASSANDRA-15078)
* Fix index summary redistribution cancellation (CASSANDRA-15045)
* Refactor Circle CI configuration (CASSANDRA-14806)
diff --git a/build.xml b/build.xml
index acd3b7adb737..3eb074fa226a 100644
--- a/build.xml
+++ b/build.xml
@@ -25,7 +25,7 @@
-
+
@@ -118,7 +118,7 @@
-
+
@@ -548,7 +548,8 @@
-
+
+
@@ -771,6 +772,7 @@
+
-
+
@@ -1168,7 +1170,6 @@
@@ -1522,6 +1523,7 @@
+
@@ -1945,6 +1947,20 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
@@ -1954,10 +1970,24 @@
-
+
-
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index ca854ca65e68..f3e5c7507c08 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -26,15 +26,21 @@ num_tokens: 256
# Triggers automatic allocation of num_tokens tokens for this node. The allocation
# algorithm attempts to choose tokens in a way that optimizes replicated load over
-# the nodes in the datacenter for the replication strategy used by the specified
-# keyspace.
+# the nodes in the datacenter for the replica factor.
#
# The load assigned to each node will be close to proportional to its number of
# vnodes.
#
# Only supported with the Murmur3Partitioner.
+
+# Replica factor is determined via the replication strategy used by the specified
+# keyspace.
# allocate_tokens_for_keyspace: KEYSPACE
+# Replica factor is explicitly set, regardless of keyspace or datacenter.
+# This is the replica factor within the datacenter, like NTS.
+# allocate_tokens_for_local_replication_factor: 3
+
# initial_token allows you to specify tokens manually. While you can use it with
# vnodes (num_tokens > 1, above) -- in which case you should provide a
# comma-separated list -- it's primarily used when adding nodes to legacy clusters
@@ -683,6 +689,16 @@ native_transport_port: 9042
# The default is true, which means all supported protocols will be honored.
native_transport_allow_older_protocols: true
+# Controls when idle client connections are closed. Idle connections are ones that had neither reads
+# nor writes for a time period.
+#
+# Clients may implement heartbeats by sending OPTIONS native protocol message after a timeout, which
+# will reset idle timeout timer on the server side. To close idle client connections, corresponding
+# values for heartbeat intervals have to be set on the client side.
+#
+# Idle connection timeouts are disabled by default.
+# native_transport_idle_timeout_in_ms: 60000
+
# The address or interface to bind the native transport server to.
#
# Set rpc_address OR rpc_interface, not both.
@@ -870,6 +886,32 @@ request_timeout_in_ms: 10000
# which picks up the OS default and configure the net.ipv4.tcp_retries2 sysctl to be ~8.
# internode_tcp_user_timeout_in_ms = 30000
+# The maximum continuous period a connection may be unwritable in application space
+# internode_application_timeout_in_ms = 30000
+
+# Global, per-endpoint and per-connection limits imposed on messages queued for delivery to other nodes
+# and waiting to be processed on arrival from other nodes in the cluster. These limits are applied to the on-wire
+# size of the message being sent or received.
+#
+# The basic per-link limit is consumed in isolation before any endpoint or global limit is imposed.
+# Each node-pair has three links: urgent, small and large. So any given node may have a maximum of
+# N*3*(internode_application_send_queue_capacity_in_bytes+internode_application_receive_queue_capacity_in_bytes)
+# messages queued without any coordination between them although in practice, with token-aware routing, only RF*tokens
+# nodes should need to communicate with significant bandwidth.
+#
+# The per-endpoint limit is imposed on all messages exceeding the per-link limit, simultaneously with the global limit,
+# on all links to or from a single node in the cluster.
+# The global limit is imposed on all messages exceeding the per-link limit, simultaneously with the per-endpoint limit,
+# on all links to or from any node in the cluster.
+#
+# internode_application_send_queue_capacity_in_bytes: 4194304 #4MiB
+# internode_application_send_queue_reserve_endpoint_capacity_in_bytes: 134217728 #128MiB
+# internode_application_send_queue_reserve_global_capacity_in_bytes: 536870912 #512MiB
+# internode_application_receive_queue_capacity_in_bytes: 4194304 #4MiB
+# internode_application_receive_queue_reserve_endpoint_capacity_in_bytes: 134217728 #128MiB
+# internode_application_receive_queue_reserve_global_capacity_in_bytes: 536870912 #512MiB
+
+
# How long before a node logs slow queries. Select queries that take longer than
# this timeout to execute, will generate an aggregated log message, so that slow queries
# can be identified. Set this value to zero to disable slow query logging.
@@ -1305,4 +1347,4 @@ enable_sasi_indexes: false
# Enables creation of transiently replicated keyspaces on this node.
# Transient replication is experimental and is not recommended for production use.
-enable_transient_replication: false
\ No newline at end of file
+enable_transient_replication: false
diff --git a/debian/changelog b/debian/changelog
index 273c6bc2ed9e..edd48a8aa0e1 100644
--- a/debian/changelog
+++ b/debian/changelog
@@ -1,8 +1,14 @@
-cassandra (4.0) UNRELEASED; urgency=medium
+cassandra (4.0~alpha2) UNRELEASED; urgency=medium
* New release
- -- Michael Shuler Wed, 15 Feb 2017 18:20:09 -0600
+ --
+
+cassandra (4.0~alpha1) unstable; urgency=medium
+
+ * New release
+
+ -- Michael Shuler Tue, 03 Sep 2019 11:51:18 -0500
cassandra (3.10) unstable; urgency=medium
diff --git a/debian/patches/cassandra_logdir_fix.diff b/debian/patches/cassandra_logdir_fix.diff
index d75553c11a37..85973313324c 100644
--- a/debian/patches/cassandra_logdir_fix.diff
+++ b/debian/patches/cassandra_logdir_fix.diff
@@ -1,14 +1,14 @@
--- a/bin/cassandra
+++ b/bin/cassandra
-@@ -171,7 +171,7 @@
- props="$3"
- class="$4"
- cassandra_parms="-Dlogback.configurationFile=logback.xml"
-- cassandra_parms="$cassandra_parms -Dcassandra.logdir=$CASSANDRA_HOME/logs"
-+ cassandra_parms="$cassandra_parms -Dcassandra.logdir=/var/log/cassandra"
- cassandra_parms="$cassandra_parms -Dcassandra.storagedir=$cassandra_storagedir"
+@@ -109,7 +109,7 @@
+ fi
+
+ if [ -z "$CASSANDRA_LOG_DIR" ]; then
+- CASSANDRA_LOG_DIR=$CASSANDRA_HOME/logs
++ CASSANDRA_LOG_DIR=/var/log/cassandra
+ fi
- if [ "x$pidpath" != "x" ]; then
+ # Special-case path variables.
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -93,16 +93,16 @@
diff --git a/doc/native_protocol_v4.spec b/doc/native_protocol_v4.spec
index 5e1e01d2dde8..5670241655f8 100644
--- a/doc/native_protocol_v4.spec
+++ b/doc/native_protocol_v4.spec
@@ -275,6 +275,9 @@ Table of Contents
mode. This mode will make all Thrift and Compact Tables to be exposed as if
they were CQL Tables. This is optional; if not specified, the option will
not be used.
+ - "THROW_ON_OVERLOAD": In case of server overloaded with too many requests, by default the server puts
+ back pressure on the client connection. Instead, the server can send an OverloadedException error message back to
+ the client if this option is set to true.
4.1.2. AUTH_RESPONSE
@@ -1185,3 +1188,4 @@ Table of Contents
* The returned in the v4 protocol is not compatible with the v3
protocol. In other words, a returned by a node using protocol v4
should not be used to query a node using protocol v3 (and vice-versa).
+ * Added THROW_ON_OVERLOAD startup option (Section 4.1.1).
diff --git a/doc/source/bugs.rst b/doc/source/bugs.rst
index bd58a8f171d3..32d676f9d3f4 100644
--- a/doc/source/bugs.rst
+++ b/doc/source/bugs.rst
@@ -18,7 +18,7 @@ Reporting Bugs
==============
If you encounter a problem with Cassandra, the first places to ask for help are the :ref:`user mailing list
-` and the ``#cassandra`` :ref:`IRC channel `.
+` and the ``cassandra`` :ref:`Slack room `.
If, after having asked for help, you suspect that you have found a bug in Cassandra, you should report it by opening a
ticket through the `Apache Cassandra JIRA `__. Please provide as much
diff --git a/doc/source/contactus.rst b/doc/source/contactus.rst
index 8d0f5dd04663..3ed9004ddcfc 100644
--- a/doc/source/contactus.rst
+++ b/doc/source/contactus.rst
@@ -17,7 +17,7 @@
Contact us
==========
-You can get in touch with the Cassandra community either via the mailing lists or the freenode IRC channels.
+You can get in touch with the Cassandra community either via the mailing lists or :ref:`Slack rooms `.
.. _mailing-lists:
@@ -39,15 +39,12 @@ Subscribe by sending an email to the email address in the Subscribe links above.
email to confirm your subscription. Make sure to keep the welcome email as it contains instructions on how to
unsubscribe.
-.. _irc-channels:
+.. _slack:
-IRC
----
+Slack
+-----
+To chat with developers or users in real-time, join our rooms on `ASF Slack `__:
-To chat with developers or users in real-time, join our channels on `IRC freenode `__. The
-following channels are available:
-
-- ``#cassandra`` - for user questions and general discussions.
-- ``#cassandra-dev`` - strictly for questions or discussions related to Cassandra development.
-- ``#cassandra-builds`` - results of automated test builds.
+- ``cassandra`` - for user questions and general discussions.
+- ``cassandra-dev`` - strictly for questions or discussions related to Cassandra development.
diff --git a/doc/source/cql/security.rst b/doc/source/cql/security.rst
index 4abeb2d1a603..429a1ef0d67d 100644
--- a/doc/source/cql/security.rst
+++ b/doc/source/cql/security.rst
@@ -148,6 +148,14 @@ status may ``DROP`` another ``SUPERUSER`` role.
Attempting to drop a role which does not exist results in an invalid query condition unless the ``IF EXISTS`` option is
used. If the option is used and the role does not exist the statement is a no-op.
+.. note:: DROP ROLE intentionally does not terminate any open user sessions. Currently connected sessions will remain
+ connected and will retain the ability to perform any database actions which do not require :ref:`authorization`.
+ However, if authorization is enabled, :ref:`permissions` of the dropped role are also revoked,
+ subject to the :ref:`caching options` configured in :ref:`cassandra.yaml`.
+ Should a dropped role be subsequently recreated and have new :ref:`permissions` or
+ :ref:`roles` granted to it, any client sessions still connected will acquire the newly granted
+ permissions and roles.
+
.. _grant-role-statement:
GRANT ROLE
diff --git a/doc/source/development/patches.rst b/doc/source/development/patches.rst
index e8e50f6a42ef..f3a2cca0f1cf 100644
--- a/doc/source/development/patches.rst
+++ b/doc/source/development/patches.rst
@@ -33,12 +33,12 @@ As a general rule of thumb:
.. hint::
- Not sure what to work? Just pick an issue tagged with the `low hanging fruit label `_ in JIRA, which we use to flag issues that could turn out to be good starter tasks for beginners.
+ Not sure what to work? Just pick an issue marked as `Low Hanging Fruit `_ Complexity in JIRA, which we use to flag issues that could turn out to be good starter tasks for beginners.
Before You Start Coding
=======================
-Although contributions are highly appreciated, we do not guarantee that each contribution will become a part of Cassandra. Therefor it's generally a good idea to first get some feedback on the things you plan to work on, especially about any new features or major changes to the code base. You can reach out to other developers on the mailing list or IRC channel listed on our `community page `_.
+Although contributions are highly appreciated, we do not guarantee that each contribution will become a part of Cassandra. Therefore it's generally a good idea to first get some feedback on the things you plan to work on, especially about any new features or major changes to the code base. You can reach out to other developers on the mailing list or :ref:`Slack `.
You should also
* Avoid redundant work by searching for already reported issues in `JIRA `_
@@ -108,7 +108,14 @@ So you've finished coding and the great moment arrives: it's time to submit your
a. Attach a patch to JIRA with a single squashed commit in it (per branch), or
b. Squash the commits in-place in your branches into one
- 6. Include a CHANGES.txt entry (put it at the top of the list), and format the commit message appropriately in your patch ending with the following statement on the last line: ``patch by X; reviewed by Y for CASSANDRA-ZZZZZ``
+ 6. Include a CHANGES.txt entry (put it at the top of the list), and format the commit message appropriately in your patch as below.
+
+ ::
+
+
+
+ patch by ; reviewed by for CASSANDRA-#####
+
7. When you're happy with the result, create a patch:
::
diff --git a/doc/source/development/release_process.rst b/doc/source/development/release_process.rst
index 23bd7abafd0a..0ab6dff1a40c 100644
--- a/doc/source/development/release_process.rst
+++ b/doc/source/development/release_process.rst
@@ -108,6 +108,15 @@ The next step is to copy and commit these binaries to staging svnpubsub::
svn add cassandra-dist-dev/
svn ci cassandra-dist-dev/
+After committing the binaries to staging, increment the version number in Cassandra on the `cassandra-`
+
+ cd ~/git/cassandra/
+ git checkout cassandra-
+ edit build.xml # update ` `
+ edit debian/changelog # add entry for new version
+ edit CHANGES.txt # add entry for new version
+ git commit -m "Update version to " build.xml debian/changelog CHANGES.txt
+ git push
Call for a Vote
===============
@@ -241,12 +250,11 @@ Fill out the following email template and send to both user and dev mailing list
[2]: (NEWS.txt) https://git1-us-west.apache.org/repos/asf?p=cassandra.git;a=blob_plain;f=NEWS.txt;hb=
[3]: https://issues.apache.org/jira/browse/CASSANDRA
-Update IRC #cassandra topic
+Update Slack Cassandra topic
---------------------------
-Update #cassandra topic on irc::
- /msg chanserv op #cassandra
- /topic #cassandra "cassandra.apache.org | Latest: 3.11.2 (https://goo.gl/M34ZbG) | Stable: 3.0.16 (https://goo.gl/B4Zumg) | Oldstable: 2.2.12 (https://goo.gl/Uf3GVw) | ask, don't ask to ask"
+Update topic in ``cassandra`` :ref:`Slack room `
+ /topic cassandra.apache.org | Latest releases: 3.11.4, 3.0.18, 2.2.14, 2.1.21 | ask, don't ask to ask
Tweet from @Cassandra
---------------------
diff --git a/doc/source/operating/compression.rst b/doc/source/operating/compression.rst
index 42a057b242d6..b4308b31a3eb 100644
--- a/doc/source/operating/compression.rst
+++ b/doc/source/operating/compression.rst
@@ -38,7 +38,9 @@ default, three options are relevant:
- ``chunk_length_in_kb`` specifies the number of kilobytes of data per compression chunk. The default is 64KB.
- ``crc_check_chance`` determines how likely Cassandra is to verify the checksum on each compression chunk during
reads. The default is 1.0.
-- ``compression_level`` is only applicable for ``ZstdCompressor`` and accepts values between ``-131072`` and ``2``.
+- ``compression_level`` is only applicable for ``ZstdCompressor`` and accepts values between ``-131072`` and ``22``.
+ The lower the level, the faster the speed (at the cost of compression). Values from 20 to 22 are called
+ "ultra levels" and should be used with caution, as they require more memory. The default is 3.
Users can set compression using the following syntax:
diff --git a/doc/source/operating/security.rst b/doc/source/operating/security.rst
index e229c7fa3e68..c2d8b79b0798 100644
--- a/doc/source/operating/security.rst
+++ b/doc/source/operating/security.rst
@@ -182,6 +182,8 @@ See also: :ref:`setting-credentials-for-internal-authentication`, :ref:`CREATE R
:ref:`ALTER ROLE `, :ref:`ALTER KEYSPACE ` and :ref:`GRANT PERMISSION
`,
+.. _authorization:
+
Authorization
^^^^^^^^^^^^^
@@ -233,6 +235,8 @@ The following assumes that authentication has already been enabled via the proce
See also: :ref:`GRANT PERMISSION `, `GRANT ALL ` and :ref:`REVOKE PERMISSION
`
+.. _auth-caching:
+
Caching
^^^^^^^
diff --git a/lib/chronicle-core-1.16.3-SNAPSHOT.jar b/lib/chronicle-core-1.16.4.jar
similarity index 63%
rename from lib/chronicle-core-1.16.3-SNAPSHOT.jar
rename to lib/chronicle-core-1.16.4.jar
index eae29e4952ee..0275a72c3018 100644
Binary files a/lib/chronicle-core-1.16.3-SNAPSHOT.jar and b/lib/chronicle-core-1.16.4.jar differ
diff --git a/lib/licenses/netty-4.1.28.txt b/lib/licenses/netty-4.1.37.txt
similarity index 100%
rename from lib/licenses/netty-4.1.28.txt
rename to lib/licenses/netty-4.1.37.txt
diff --git a/lib/licenses/netty-tcnative-2.0.25.txt b/lib/licenses/netty-tcnative-2.0.25.txt
new file mode 100644
index 000000000000..261eeb9e9f8b
--- /dev/null
+++ b/lib/licenses/netty-tcnative-2.0.25.txt
@@ -0,0 +1,201 @@
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
diff --git a/lib/netty-all-4.1.28.Final.jar b/lib/netty-all-4.1.37.Final.jar
similarity index 56%
rename from lib/netty-all-4.1.28.Final.jar
rename to lib/netty-all-4.1.37.Final.jar
index 058662ecc9f4..93cff04768e6 100644
Binary files a/lib/netty-all-4.1.28.Final.jar and b/lib/netty-all-4.1.37.Final.jar differ
diff --git a/lib/netty-tcnative-boringssl-static-2.0.25.Final.jar b/lib/netty-tcnative-boringssl-static-2.0.25.Final.jar
new file mode 100644
index 000000000000..954627fb73f6
Binary files /dev/null and b/lib/netty-tcnative-boringssl-static-2.0.25.Final.jar differ
diff --git a/redhat/cassandra.spec b/redhat/cassandra.spec
index eaf7565922f5..ca5d38e3e7d5 100644
--- a/redhat/cassandra.spec
+++ b/redhat/cassandra.spec
@@ -8,7 +8,9 @@
%global username cassandra
-%define relname apache-cassandra-%{version}
+# input of ~alphaN, ~betaN, ~rcN package versions need to retain upstream '-alphaN, etc' version for sources
+%define upstream_version %(echo %{version} | sed -r 's/~/-/g')
+%define relname apache-cassandra-%{upstream_version}
Name: cassandra
Version: %{version}
@@ -74,14 +76,14 @@ patch -p1 < debian/patches/cassandra_logdir_fix.diff
sed -i 's/^# hints_directory:/hints_directory:/' conf/cassandra.yaml
# remove batch, powershell, and other files not being installed
-rm conf/*.ps1
-rm bin/*.bat
-rm bin/*.orig
-rm bin/*.ps1
-rm bin/cassandra.in.sh
-rm lib/sigar-bin/*winnt* # strip segfaults on dll..
-rm tools/bin/*.bat
-rm tools/bin/cassandra.in.sh
+rm -f conf/*.ps1
+rm -f bin/*.bat
+rm -f bin/*.orig
+rm -f bin/*.ps1
+rm -f bin/cassandra.in.sh
+rm -f lib/sigar-bin/*winnt* # strip segfaults on dll..
+rm -f tools/bin/*.bat
+rm -f tools/bin/cassandra.in.sh
# copy default configs
cp -pr conf/* %{buildroot}/%{_sysconfdir}/%{username}/default.conf/
@@ -118,10 +120,12 @@ exit 0
%files
%defattr(0644,root,root,0755)
%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt
+%attr(755,root,root) %{_bindir}/auditlogviewer
%attr(755,root,root) %{_bindir}/cassandra-stress
%attr(755,root,root) %{_bindir}/cqlsh
%attr(755,root,root) %{_bindir}/cqlsh.py
%attr(755,root,root) %{_bindir}/debug-cql
+%attr(755,root,root) %{_bindir}/fqltool
%attr(755,root,root) %{_bindir}/nodetool
%attr(755,root,root) %{_bindir}/sstableloader
%attr(755,root,root) %{_bindir}/sstablescrub
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index 238b5b506e79..37ad60a9e4cd 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -53,7 +53,7 @@ public class CassandraAuthorizer implements IAuthorizer
private static final String RESOURCE = "resource";
private static final String PERMISSIONS = "permissions";
- SelectStatement authorizeRoleStatement;
+ private SelectStatement authorizeRoleStatement;
public CassandraAuthorizer()
{
@@ -63,16 +63,24 @@ public CassandraAuthorizer()
// or indirectly via roles granted to the user.
public Set authorize(AuthenticatedUser user, IResource resource)
{
- if (user.isSuper())
- return resource.applicablePermissions();
+ try
+ {
+ if (user.isSuper())
+ return resource.applicablePermissions();
- Set permissions = EnumSet.noneOf(Permission.class);
+ Set permissions = EnumSet.noneOf(Permission.class);
- // Even though we only care about the RoleResource here, we use getRoleDetails as
- // it saves a Set creation in RolesCache
- for (Role role: user.getRoleDetails())
- addPermissionsForRole(permissions, resource, role.resource);
- return permissions;
+ // Even though we only care about the RoleResource here, we use getRoleDetails as
+ // it saves a Set creation in RolesCache
+ for (Role role: user.getRoleDetails())
+ addPermissionsForRole(permissions, resource, role.resource);
+ return permissions;
+ }
+ catch (RequestExecutionException | RequestValidationException e)
+ {
+ logger.debug("Failed to authorize {} for {}", user, resource);
+ throw new UnauthorizedException("Unable to perform authorization of permissions: " + e.getMessage(), e);
+ }
}
public void grant(AuthenticatedUser performer, Set permissions, IResource resource, RoleResource grantee)
diff --git a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
index 34a01402683b..6fdcd6959c9d 100644
--- a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
@@ -78,7 +78,7 @@ private Set getAuthorizedDcs(String name)
public DCPermissions authorize(RoleResource role)
{
- if (!DatabaseDescriptor.getRoleManager().canLogin(role))
+ if (!Roles.canLogin(role))
{
return DCPermissions.none();
}
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index ebb7d5ff99d4..f4c942859245 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -270,12 +270,28 @@ public Set getAllRoles() throws RequestValidationException, Reques
public boolean isSuper(RoleResource role)
{
- return getRole(role.getRoleName()).isSuper;
+ try
+ {
+ return getRole(role.getRoleName()).isSuper;
+ }
+ catch (RequestExecutionException e)
+ {
+ logger.debug("Failed to authorize {} for super-user permission", role.getRoleName());
+ throw new UnauthorizedException("Unable to perform authorization of super-user permission: " + e.getMessage(), e);
+ }
}
public boolean canLogin(RoleResource role)
{
- return getRole(role.getRoleName()).canLogin;
+ try
+ {
+ return getRole(role.getRoleName()).canLogin;
+ }
+ catch (RequestExecutionException e)
+ {
+ logger.debug("Failed to authorize {} for login permission", role.getRoleName());
+ throw new UnauthorizedException("Unable to perform authorization of login permission: " + e.getMessage(), e);
+ }
}
public Map getCustomOptions(RoleResource role)
diff --git a/src/java/org/apache/cassandra/auth/IAuthenticator.java b/src/java/org/apache/cassandra/auth/IAuthenticator.java
index 212e77495052..80ea719237b9 100644
--- a/src/java/org/apache/cassandra/auth/IAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/IAuthenticator.java
@@ -105,7 +105,7 @@ default SaslNegotiator newSaslNegotiator(InetAddress clientAddress, X509Certific
public interface SaslNegotiator
{
/**
- * Evaluates the client response data and generates a byte[] reply which may be a further challenge or purely
+ * Evaluates the client response data and generates a byte[] response which may be a further challenge or purely
* informational in the case that the negotiation is completed on this round.
*
* This method is called each time a {@link org.apache.cassandra.transport.messages.AuthResponse} is received
diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
index 27a68a02c043..9da99a9a6083 100644
--- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
@@ -29,6 +29,7 @@
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.RequestExecutionException;
import org.apache.cassandra.schema.SchemaConstants;
import org.apache.cassandra.cql3.QueryOptions;
import org.apache.cassandra.cql3.QueryProcessor;
@@ -64,7 +65,7 @@ public class PasswordAuthenticator implements IAuthenticator
public static final String USERNAME_KEY = "username";
public static final String PASSWORD_KEY = "password";
- private static final byte NUL = 0;
+ static final byte NUL = 0;
private SelectStatement authenticateStatement;
private CredentialsCache cache;
@@ -100,23 +101,30 @@ private AuthenticatedUser authenticate(String username, String password) throws
private String queryHashedPassword(String username)
{
- ResultMessage.Rows rows =
- authenticateStatement.execute(QueryState.forInternalCalls(),
- QueryOptions.forInternalCalls(consistencyForRole(username),
- Lists.newArrayList(ByteBufferUtil.bytes(username))),
- System.nanoTime());
-
- // If either a non-existent role name was supplied, or no credentials
- // were found for that role we don't want to cache the result so we throw
- // a specific, but unchecked, exception to keep LoadingCache happy.
- if (rows.result.isEmpty())
- throw new AuthenticationException(String.format("Provided username %s and/or password are incorrect", username));
-
- UntypedResultSet result = UntypedResultSet.create(rows.result);
- if (!result.one().has(SALTED_HASH))
- throw new AuthenticationException(String.format("Provided username %s and/or password are incorrect", username));
-
- return result.one().getString(SALTED_HASH);
+ try
+ {
+ ResultMessage.Rows rows =
+ authenticateStatement.execute(QueryState.forInternalCalls(),
+ QueryOptions.forInternalCalls(consistencyForRole(username),
+ Lists.newArrayList(ByteBufferUtil.bytes(username))),
+ System.nanoTime());
+
+ // If either a non-existent role name was supplied, or no credentials
+ // were found for that role we don't want to cache the result so we throw
+ // an exception.
+ if (rows.result.isEmpty())
+ throw new AuthenticationException(String.format("Provided username %s and/or password are incorrect", username));
+
+ UntypedResultSet result = UntypedResultSet.create(rows.result);
+ if (!result.one().has(SALTED_HASH))
+ throw new AuthenticationException(String.format("Provided username %s and/or password are incorrect", username));
+
+ return result.one().getString(SALTED_HASH);
+ }
+ catch (RequestExecutionException e)
+ {
+ throw new AuthenticationException("Unable to perform authentication: " + e.getMessage(), e);
+ }
}
public Set protectedResources()
@@ -206,7 +214,7 @@ private void decodeCredentials(byte[] bytes) throws AuthenticationException
byte[] user = null;
byte[] pass = null;
int end = bytes.length;
- for (int i = bytes.length - 1 ; i >= 0; i--)
+ for (int i = bytes.length - 1; i >= 0; i--)
{
if (bytes[i] == NUL)
{
@@ -214,13 +222,16 @@ private void decodeCredentials(byte[] bytes) throws AuthenticationException
pass = Arrays.copyOfRange(bytes, i + 1, end);
else if (user == null)
user = Arrays.copyOfRange(bytes, i + 1, end);
+ else
+ throw new AuthenticationException("Credential format error: username or password is empty or contains NUL(\\0) character");
+
end = i;
}
}
- if (pass == null)
+ if (pass == null || pass.length == 0)
throw new AuthenticationException("Password must not be null");
- if (user == null)
+ if (user == null || user.length == 0)
throw new AuthenticationException("Authentication ID must not be null");
username = new String(user, StandardCharsets.UTF_8);
diff --git a/src/java/org/apache/cassandra/auth/Roles.java b/src/java/org/apache/cassandra/auth/Roles.java
index 22eb0d31e82a..527451e703c0 100644
--- a/src/java/org/apache/cassandra/auth/Roles.java
+++ b/src/java/org/apache/cassandra/auth/Roles.java
@@ -25,10 +25,17 @@
import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
public class Roles
{
+ private static final Logger logger = LoggerFactory.getLogger(Roles.class);
+
private static final Role NO_ROLE = new Role("", false, false, Collections.emptyMap(), Collections.emptySet());
private static RolesCache cache;
@@ -91,11 +98,19 @@ public static Set getRoleDetails(RoleResource primaryRole)
*/
public static boolean hasSuperuserStatus(RoleResource role)
{
- for (Role r : getRoleDetails(role))
- if (r.isSuper)
- return true;
-
- return false;
+ try
+ {
+ for (Role r : getRoleDetails(role))
+ if (r.isSuper)
+ return true;
+
+ return false;
+ }
+ catch (RequestExecutionException e)
+ {
+ logger.debug("Failed to authorize {} for super-user permission", role.getRoleName());
+ throw new UnauthorizedException("Unable to perform authorization of super-user permission: " + e.getMessage(), e);
+ }
}
/**
@@ -106,11 +121,19 @@ public static boolean hasSuperuserStatus(RoleResource role)
*/
public static boolean canLogin(final RoleResource role)
{
- for (Role r : getRoleDetails(role))
- if (r.resource.equals(role))
- return r.canLogin;
-
- return false;
+ try
+ {
+ for (Role r : getRoleDetails(role))
+ if (r.resource.equals(role))
+ return r.canLogin;
+
+ return false;
+ }
+ catch (RequestExecutionException e)
+ {
+ logger.debug("Failed to authorize {} for login permission", role.getRoleName());
+ throw new UnauthorizedException("Unable to perform authorization of login permission: " + e.getMessage(), e);
+ }
}
/**
diff --git a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
index ef00027604b9..b213c43d66c2 100644
--- a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
+++ b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
@@ -33,7 +33,6 @@
import javax.security.auth.Subject;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -491,10 +490,5 @@ protected JMXPermissionsCache()
AuthorizationProxy::loadPermissions,
() -> true);
}
-
- public Set get(RoleResource roleResource)
- {
- return super.get(roleResource);
- }
}
}
diff --git a/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java b/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java
index 3c3fcec49096..3443cab78393 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java
@@ -20,11 +20,13 @@
import java.util.UUID;
import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
public final class BatchRemoveVerbHandler implements IVerbHandler
{
- public void doVerb(MessageIn message, int id)
+ public static final BatchRemoveVerbHandler instance = new BatchRemoveVerbHandler();
+
+ public void doVerb(Message message)
{
BatchlogManager.remove(message.payload);
}
diff --git a/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java b/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java
index 4bc878cbf592..77335cb44389 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java
@@ -17,16 +17,17 @@
*/
package org.apache.cassandra.batchlog;
-import org.apache.cassandra.db.WriteResponse;
import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
import org.apache.cassandra.net.MessagingService;
public final class BatchStoreVerbHandler implements IVerbHandler
{
- public void doVerb(MessageIn message, int id)
+ public static final BatchStoreVerbHandler instance = new BatchStoreVerbHandler();
+
+ public void doVerb(Message message)
{
BatchlogManager.store(message.payload);
- MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from);
+ MessagingService.instance().send(message.emptyResponse(), message.from());
}
}
diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
index b2b851df89d0..f140332f509f 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -32,6 +32,7 @@
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Collections2;
@@ -65,20 +66,23 @@
import org.apache.cassandra.locator.ReplicaLayout;
import org.apache.cassandra.locator.ReplicaPlan;
import org.apache.cassandra.locator.Replicas;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessageFlag;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.schema.SchemaConstants;
import org.apache.cassandra.schema.TableId;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.service.WriteResponseHandler;
+import org.apache.cassandra.utils.ExecutorUtils;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.MBeanWrapper;
import org.apache.cassandra.utils.UUIDGen;
import static com.google.common.collect.Iterables.transform;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
import static org.apache.cassandra.cql3.QueryProcessor.executeInternalWithPaging;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
public class BatchlogManager implements BatchlogManagerMBean
{
@@ -88,7 +92,7 @@ public class BatchlogManager implements BatchlogManagerMBean
private static final Logger logger = LoggerFactory.getLogger(BatchlogManager.class);
public static final BatchlogManager instance = new BatchlogManager();
- public static final long BATCHLOG_REPLAY_TIMEOUT = Long.getLong("cassandra.batchlog.replay_timeout_in_ms", DatabaseDescriptor.getWriteRpcTimeout() * 2);
+ public static final long BATCHLOG_REPLAY_TIMEOUT = Long.getLong("cassandra.batchlog.replay_timeout_in_ms", DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS) * 2);
private volatile long totalBatchesReplayed = 0; // no concurrency protection necessary as only written by replay thread.
private volatile UUID lastReplayedUuid = UUIDGen.minTimeUUID(0);
@@ -112,13 +116,12 @@ public void start()
batchlogTasks.scheduleWithFixedDelay(this::replayFailedBatches,
StorageService.RING_DELAY,
REPLAY_INTERVAL,
- TimeUnit.MILLISECONDS);
+ MILLISECONDS);
}
- public void shutdown() throws InterruptedException
+ public void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
{
- batchlogTasks.shutdown();
- batchlogTasks.awaitTermination(60, TimeUnit.SECONDS);
+ ExecutorUtils.shutdownAndWait(timeout, unit, batchlogTasks);
}
public static void remove(UUID id)
@@ -356,7 +359,7 @@ public int replay(RateLimiter rateLimiter, Set hintedNodes)
return 0;
int gcgs = gcgs(mutations);
- if (TimeUnit.MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds())
+ if (MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds())
return 0;
replayHandlers = sendReplays(mutations, writtenAt, hintedNodes);
@@ -419,7 +422,7 @@ private void writeHintsForUndeliveredEndpoints(int startFrom, Set sendSingleReplayMutation(fin
ReplicaPlan.ForTokenWrite replicaPlan = new ReplicaPlan.ForTokenWrite(keyspace, ConsistencyLevel.ONE,
liveRemoteOnly.pending(), liveRemoteOnly.all(), liveRemoteOnly.all(), liveRemoteOnly.all());
ReplayWriteResponseHandler handler = new ReplayWriteResponseHandler<>(replicaPlan, System.nanoTime());
- MessageOut message = mutation.createMessage();
+ Message message = Message.outWithFlag(MUTATION_REQ, mutation, MessageFlag.CALL_BACK_ON_FAILURE);
for (Replica replica : liveRemoteOnly.all())
- MessagingService.instance().sendWriteRR(message, replica, handler, false);
+ MessagingService.instance().sendWriteWithCallback(message, replica, handler, false);
return handler;
}
@@ -506,7 +509,7 @@ private static int gcgs(Collection mutations)
/**
* A wrapper of WriteResponseHandler that stores the addresses of the endpoints from
- * which we did not receive a successful reply.
+ * which we did not receive a successful response.
*/
private static class ReplayWriteResponseHandler extends WriteResponseHandler
{
@@ -525,11 +528,11 @@ protected int blockFor()
}
@Override
- public void response(MessageIn m)
+ public void onResponse(Message m)
{
- boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddressAndPort() : m.from);
+ boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddressAndPort() : m.from());
assert removed;
- super.response(m);
+ super.onResponse(m);
}
}
}
diff --git a/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java
new file mode 100644
index 000000000000..1a00e4f3beda
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java
@@ -0,0 +1,56 @@
+/*
+ * 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.concurrent;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.AbstractExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ImmediateExecutor extends AbstractExecutorService implements LocalAwareExecutorService
+{
+ public static final ImmediateExecutor INSTANCE = new ImmediateExecutor();
+
+ private ImmediateExecutor() {}
+
+ public void execute(Runnable command, ExecutorLocals locals)
+ {
+ command.run();
+ }
+
+ public void maybeExecuteImmediately(Runnable command)
+ {
+ command.run();
+ }
+
+ public void execute(Runnable command)
+ {
+ command.run();
+ }
+
+ public int getActiveTaskCount() { return 0; }
+ public long getCompletedTaskCount() { return 0; }
+ public int getPendingTaskCount() { return 0; }
+ public int getMaximumPoolSize() { return 0; }
+ public void shutdown() { }
+ public List shutdownNow() { return Collections.emptyList(); }
+ public boolean isShutdown() { return false; }
+ public boolean isTerminated() { return false; }
+ public boolean awaitTermination(long timeout, TimeUnit unit) { return true; }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
index 199803f04489..b54fa3fca51f 100644
--- a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
@@ -70,7 +70,7 @@ public InfiniteLoopExecutor start()
return this;
}
- public void shutdown()
+ public void shutdownNow()
{
isShutdown = true;
thread.interrupt();
diff --git a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
index 33f1312b9f8c..7cc73bd1c3c4 100644
--- a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
+++ b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
@@ -24,6 +24,7 @@
import io.netty.util.concurrent.FastThreadLocal;
import io.netty.util.concurrent.FastThreadLocalThread;
+import org.apache.cassandra.utils.memory.BufferPool;
/**
* This class is an implementation of the ThreadFactory interface. This
@@ -35,6 +36,7 @@ public class NamedThreadFactory implements ThreadFactory
{
private static volatile String globalPrefix;
public static void setGlobalPrefix(String prefix) { globalPrefix = prefix; }
+ public static String globalPrefix() { return globalPrefix == null ? "" : globalPrefix; }
public final String id;
private final int priority;
diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
index 5e3e5cf3964c..c549c4d587c8 100644
--- a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
+++ b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
@@ -17,10 +17,17 @@
*/
package org.apache.cassandra.concurrent;
+import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.utils.ExecutorUtils;
+
+import org.apache.cassandra.utils.ExecutorUtils;
/**
* Centralized location for shared executors
@@ -48,12 +55,8 @@ public class ScheduledExecutors
public static final DebuggableScheduledThreadPoolExecutor optionalTasks = new DebuggableScheduledThreadPoolExecutor("OptionalTasks");
@VisibleForTesting
- public static void shutdownAndWait() throws InterruptedException
+ public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
{
- ExecutorService[] executors = new ExecutorService[] { scheduledFastTasks, scheduledTasks, nonPeriodicTasks, optionalTasks };
- for (ExecutorService executor : executors)
- executor.shutdownNow();
- for (ExecutorService executor : executors)
- executor.awaitTermination(60, TimeUnit.SECONDS);
+ ExecutorUtils.shutdownNowAndWait(timeout, unit, scheduledFastTasks, scheduledTasks, nonPeriodicTasks, optionalTasks);
}
}
diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
index 62bede9add1a..3388ea495807 100644
--- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
+++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
@@ -22,6 +22,7 @@
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.LockSupport;
@@ -107,14 +108,14 @@ void maybeStartSpinningWorker()
schedule(Work.SPINNING);
}
- public LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
+ public synchronized LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
{
SEPExecutor executor = new SEPExecutor(this, maxConcurrency, maxQueuedTasks, jmxPath, name);
executors.add(executor);
return executor;
}
- public void shutdown() throws InterruptedException
+ public synchronized void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
{
shuttingDown = true;
for (SEPExecutor executor : executors)
@@ -122,9 +123,13 @@ public void shutdown() throws InterruptedException
terminateWorkers();
- long until = System.nanoTime() + TimeUnit.MINUTES.toNanos(1L);
+ long until = System.nanoTime() + unit.toNanos(timeout);
for (SEPExecutor executor : executors)
+ {
executor.shutdown.await(until - System.nanoTime(), TimeUnit.NANOSECONDS);
+ if (!executor.isTerminated())
+ throw new TimeoutException(executor.name + " not terminated");
+ }
}
void terminateWorkers()
diff --git a/src/java/org/apache/cassandra/concurrent/Stage.java b/src/java/org/apache/cassandra/concurrent/Stage.java
index ccb156501e40..ed13eebe2b47 100644
--- a/src/java/org/apache/cassandra/concurrent/Stage.java
+++ b/src/java/org/apache/cassandra/concurrent/Stage.java
@@ -17,11 +17,6 @@
*/
package org.apache.cassandra.concurrent;
-import java.util.Arrays;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-
public enum Stage
{
READ,
@@ -35,18 +30,7 @@ public enum Stage
MISC,
TRACING,
INTERNAL_RESPONSE,
- READ_REPAIR;
-
- public static Iterable jmxEnabledStages()
- {
- return Iterables.filter(Arrays.asList(values()), new Predicate()
- {
- public boolean apply(Stage stage)
- {
- return stage != TRACING;
- }
- });
- }
+ IMMEDIATE;
public String getJmxType()
{
@@ -58,13 +42,13 @@ public String getJmxType()
case MISC:
case TRACING:
case INTERNAL_RESPONSE:
+ case IMMEDIATE:
return "internal";
case MUTATION:
case COUNTER_MUTATION:
case VIEW_MUTATION:
case READ:
case REQUEST_RESPONSE:
- case READ_REPAIR:
return "request";
default:
throw new AssertionError("Unknown stage " + this);
diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java
index 608a00595216..b0d34aee09bc 100644
--- a/src/java/org/apache/cassandra/concurrent/StageManager.java
+++ b/src/java/org/apache/cassandra/concurrent/StageManager.java
@@ -17,7 +17,9 @@
*/
package org.apache.cassandra.concurrent;
+import java.util.Collections;
import java.util.EnumMap;
+import java.util.List;
import java.util.concurrent.*;
import com.google.common.annotations.VisibleForTesting;
@@ -25,9 +27,12 @@
import org.slf4j.LoggerFactory;
import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.utils.ExecutorUtils;
import org.apache.cassandra.utils.FBUtilities;
import static org.apache.cassandra.config.DatabaseDescriptor.*;
+import static org.apache.cassandra.utils.ExecutorUtils.*;
/**
@@ -56,24 +61,18 @@ public class StageManager
stages.put(Stage.ANTI_ENTROPY, new JMXEnabledThreadPoolExecutor(Stage.ANTI_ENTROPY));
stages.put(Stage.MIGRATION, new JMXEnabledThreadPoolExecutor(Stage.MIGRATION));
stages.put(Stage.MISC, new JMXEnabledThreadPoolExecutor(Stage.MISC));
- stages.put(Stage.READ_REPAIR, multiThreadedStage(Stage.READ_REPAIR, FBUtilities.getAvailableProcessors()));
stages.put(Stage.TRACING, tracingExecutor());
+ stages.put(Stage.IMMEDIATE, ImmediateExecutor.INSTANCE);
}
private static LocalAwareExecutorService tracingExecutor()
{
- RejectedExecutionHandler reh = new RejectedExecutionHandler()
- {
- public void rejectedExecution(Runnable r, ThreadPoolExecutor executor)
- {
- MessagingService.instance().incrementDroppedMessages(MessagingService.Verb._TRACE);
- }
- };
+ RejectedExecutionHandler reh = (r, executor) -> MessagingService.instance().metrics.recordSelfDroppedMessage(Verb._TRACE);
return new TracingExecutor(1,
1,
KEEPALIVE,
TimeUnit.SECONDS,
- new ArrayBlockingQueue(1000),
+ new ArrayBlockingQueue<>(1000),
new NamedThreadFactory(Stage.TRACING.getJmxName()),
reh);
}
@@ -114,12 +113,9 @@ public static void shutdownNow()
}
@VisibleForTesting
- public static void shutdownAndWait() throws InterruptedException
+ public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
{
- for (Stage stage : Stage.values())
- StageManager.stages.get(stage).shutdown();
- for (Stage stage : Stage.values())
- StageManager.stages.get(stage).awaitTermination(60, TimeUnit.SECONDS);
+ ExecutorUtils.shutdownNowAndWait(timeout, unit, StageManager.stages.values());
}
/**
@@ -155,4 +151,5 @@ public int getPendingTaskCount()
return getQueue().size();
}
}
+
}
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index a6050bea8baa..b86b7c57e690 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -86,6 +86,10 @@ public class Config
public int num_tokens = 1;
/** Triggers automatic allocation of tokens if set, using the replication strategy of the referenced keyspace */
public String allocate_tokens_for_keyspace = null;
+ /** Triggers automatic allocation of tokens if set, based on the provided replica count for a datacenter */
+ public Integer allocate_tokens_for_local_replication_factor = null;
+
+ public long native_transport_idle_timeout_in_ms = 0L;
public volatile long request_timeout_in_ms = 10000L;
@@ -128,6 +132,8 @@ public class Config
public volatile Integer repair_session_max_tree_depth = null;
public volatile Integer repair_session_space_in_mb = null;
+ public volatile boolean use_offheap_merkle_trees = true;
+
public int storage_port = 7000;
public int ssl_storage_port = 7001;
public String listen_address;
@@ -149,8 +155,21 @@ public class Config
public boolean rpc_interface_prefer_ipv6 = false;
public String broadcast_rpc_address;
public boolean rpc_keepalive = true;
- public int internode_send_buff_size_in_bytes = 0;
- public int internode_recv_buff_size_in_bytes = 0;
+
+ public Integer internode_max_message_size_in_bytes;
+
+ public int internode_socket_send_buffer_size_in_bytes = 0;
+ public int internode_socket_receive_buffer_size_in_bytes = 0;
+
+ // TODO: derive defaults from system memory settings?
+ public int internode_application_send_queue_capacity_in_bytes = 1 << 22; // 4MiB
+ public int internode_application_send_queue_reserve_endpoint_capacity_in_bytes = 1 << 27; // 128MiB
+ public int internode_application_send_queue_reserve_global_capacity_in_bytes = 1 << 29; // 512MiB
+
+ public int internode_application_receive_queue_capacity_in_bytes = 1 << 22; // 4MiB
+ public int internode_application_receive_queue_reserve_endpoint_capacity_in_bytes = 1 << 27; // 128MiB
+ public int internode_application_receive_queue_reserve_global_capacity_in_bytes = 1 << 29; // 512MiB
+
// Defensive settings for protecting Cassandra from true network partitions. See (CASSANDRA-14358) for details.
// The amount of time to wait for internode tcp connections to establish.
public int internode_tcp_connect_timeout_in_ms = 2000;
@@ -170,6 +189,9 @@ public class Config
public boolean native_transport_flush_in_batches_legacy = false;
public volatile boolean native_transport_allow_older_protocols = true;
public int native_transport_frame_block_size_in_kb = 32;
+ public volatile long native_transport_max_concurrent_requests_in_bytes_per_ip = -1L;
+ public volatile long native_transport_max_concurrent_requests_in_bytes = -1L;
+
/**
* Max size of values in SSTables, in MegaBytes.
@@ -322,7 +344,7 @@ public class Config
public volatile ConsistencyLevel ideal_consistency_level = null;
/*
- * Strategy to use for coalescing messages in {@link OutboundMessagingPool}.
+ * Strategy to use for coalescing messages in {@link OutboundConnections}.
* Can be fixed, movingaverage, timehorizon, disabled. Setting is case and leading/trailing
* whitespace insensitive. You can also specify a subclass of
* {@link org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy} by name.
@@ -339,12 +361,6 @@ public class Config
public int otc_coalescing_window_us = otc_coalescing_window_us_default;
public int otc_coalescing_enough_coalesced_messages = 8;
- /**
- * Backlog expiration interval in milliseconds for the OutboundTcpConnection.
- */
- public static final int otc_backlog_expiration_interval_ms_default = 200;
- public volatile int otc_backlog_expiration_interval_ms = otc_backlog_expiration_interval_ms_default;
-
public int windows_timer_interval = 0;
/**
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index b3ab054ce95a..e4ea611b734f 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -27,6 +27,7 @@
import java.nio.file.Paths;
import java.util.*;
import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
import java.util.function.Supplier;
import com.google.common.annotations.VisibleForTesting;
@@ -72,6 +73,7 @@
import org.apache.commons.lang3.StringUtils;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
public class DatabaseDescriptor
@@ -80,6 +82,7 @@ public class DatabaseDescriptor
{
// This static block covers most usages
FBUtilities.preventIllegalAccessWarnings();
+ System.setProperty("io.netty.transport.estimateSizeOnSubmit", "false");
}
private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class);
@@ -505,6 +508,16 @@ else if (conf.native_transport_max_frame_size_in_mb >= 2048)
conf.hints_directory = storagedirFor("hints");
}
+ if (conf.native_transport_max_concurrent_requests_in_bytes <= 0)
+ {
+ conf.native_transport_max_concurrent_requests_in_bytes = Runtime.getRuntime().maxMemory() / 10;
+ }
+
+ if (conf.native_transport_max_concurrent_requests_in_bytes_per_ip <= 0)
+ {
+ conf.native_transport_max_concurrent_requests_in_bytes_per_ip = Runtime.getRuntime().maxMemory() / 40;
+ }
+
if (conf.cdc_raw_directory == null)
{
conf.cdc_raw_directory = storagedirFor("cdc_raw");
@@ -801,6 +814,28 @@ else if (conf.max_value_size_in_mb >= 2048)
if (conf.otc_coalescing_enough_coalesced_messages <= 0)
throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false);
+ Integer maxMessageSize = conf.internode_max_message_size_in_bytes;
+ if (maxMessageSize != null)
+ {
+ if (maxMessageSize > conf.internode_application_receive_queue_reserve_endpoint_capacity_in_bytes)
+ throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_receive_queue_reserve_endpoint_capacity_in_bytes", false);
+
+ if (maxMessageSize > conf.internode_application_receive_queue_reserve_global_capacity_in_bytes)
+ throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_receive_queue_reserve_global_capacity_in_bytes", false);
+
+ if (maxMessageSize > conf.internode_application_send_queue_reserve_endpoint_capacity_in_bytes)
+ throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_send_queue_reserve_endpoint_capacity_in_bytes", false);
+
+ if (maxMessageSize > conf.internode_application_send_queue_reserve_global_capacity_in_bytes)
+ throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_send_queue_reserve_global_capacity_in_bytes", false);
+ }
+ else
+ {
+ conf.internode_max_message_size_in_bytes =
+ Math.min(conf.internode_application_receive_queue_reserve_endpoint_capacity_in_bytes,
+ conf.internode_application_send_queue_reserve_endpoint_capacity_in_bytes);
+ }
+
validateMaxConcurrentAutoUpgradeTasksConf(conf.max_concurrent_automatic_sstable_upgrades);
}
@@ -1387,6 +1422,11 @@ public static String getAllocateTokensForKeyspace()
return System.getProperty(Config.PROPERTY_PREFIX + "allocate_tokens_for_keyspace", conf.allocate_tokens_for_keyspace);
}
+ public static Integer getAllocateTokensForLocalRf()
+ {
+ return conf.allocate_tokens_for_local_replication_factor;
+ }
+
public static Collection tokensFromString(String tokenString)
{
List tokens = new ArrayList();
@@ -1448,9 +1488,19 @@ public static int getSSLStoragePort()
return Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "ssl_storage_port", Integer.toString(conf.ssl_storage_port)));
}
- public static long getRpcTimeout()
+ public static long nativeTransportIdleTimeout()
+ {
+ return conf.native_transport_idle_timeout_in_ms;
+ }
+
+ public static void setNativeTransportIdleTimeout(long nativeTransportTimeout)
+ {
+ conf.native_transport_idle_timeout_in_ms = nativeTransportTimeout;
+ }
+
+ public static long getRpcTimeout(TimeUnit unit)
{
- return conf.request_timeout_in_ms;
+ return unit.convert(conf.request_timeout_in_ms, MILLISECONDS);
}
public static void setRpcTimeout(long timeOutInMillis)
@@ -1458,9 +1508,9 @@ public static void setRpcTimeout(long timeOutInMillis)
conf.request_timeout_in_ms = timeOutInMillis;
}
- public static long getReadRpcTimeout()
+ public static long getReadRpcTimeout(TimeUnit unit)
{
- return conf.read_request_timeout_in_ms;
+ return unit.convert(conf.read_request_timeout_in_ms, MILLISECONDS);
}
public static void setReadRpcTimeout(long timeOutInMillis)
@@ -1468,9 +1518,9 @@ public static void setReadRpcTimeout(long timeOutInMillis)
conf.read_request_timeout_in_ms = timeOutInMillis;
}
- public static long getRangeRpcTimeout()
+ public static long getRangeRpcTimeout(TimeUnit unit)
{
- return conf.range_request_timeout_in_ms;
+ return unit.convert(conf.range_request_timeout_in_ms, MILLISECONDS);
}
public static void setRangeRpcTimeout(long timeOutInMillis)
@@ -1478,9 +1528,9 @@ public static void setRangeRpcTimeout(long timeOutInMillis)
conf.range_request_timeout_in_ms = timeOutInMillis;
}
- public static long getWriteRpcTimeout()
+ public static long getWriteRpcTimeout(TimeUnit unit)
{
- return conf.write_request_timeout_in_ms;
+ return unit.convert(conf.write_request_timeout_in_ms, MILLISECONDS);
}
public static void setWriteRpcTimeout(long timeOutInMillis)
@@ -1488,9 +1538,9 @@ public static void setWriteRpcTimeout(long timeOutInMillis)
conf.write_request_timeout_in_ms = timeOutInMillis;
}
- public static long getCounterWriteRpcTimeout()
+ public static long getCounterWriteRpcTimeout(TimeUnit unit)
{
- return conf.counter_write_request_timeout_in_ms;
+ return unit.convert(conf.counter_write_request_timeout_in_ms, MILLISECONDS);
}
public static void setCounterWriteRpcTimeout(long timeOutInMillis)
@@ -1498,9 +1548,9 @@ public static void setCounterWriteRpcTimeout(long timeOutInMillis)
conf.counter_write_request_timeout_in_ms = timeOutInMillis;
}
- public static long getCasContentionTimeout()
+ public static long getCasContentionTimeout(TimeUnit unit)
{
- return conf.cas_contention_timeout_in_ms;
+ return unit.convert(conf.cas_contention_timeout_in_ms, MILLISECONDS);
}
public static void setCasContentionTimeout(long timeOutInMillis)
@@ -1508,9 +1558,9 @@ public static void setCasContentionTimeout(long timeOutInMillis)
conf.cas_contention_timeout_in_ms = timeOutInMillis;
}
- public static long getTruncateRpcTimeout()
+ public static long getTruncateRpcTimeout(TimeUnit unit)
{
- return conf.truncate_request_timeout_in_ms;
+ return unit.convert(conf.truncate_request_timeout_in_ms, MILLISECONDS);
}
public static void setTruncateRpcTimeout(long timeOutInMillis)
@@ -1523,27 +1573,32 @@ public static boolean hasCrossNodeTimeout()
return conf.cross_node_timeout;
}
- public static long getSlowQueryTimeout()
+ public static void setCrossNodeTimeout(boolean crossNodeTimeout)
{
- return conf.slow_query_log_timeout_in_ms;
+ conf.cross_node_timeout = crossNodeTimeout;
+ }
+
+ public static long getSlowQueryTimeout(TimeUnit units)
+ {
+ return units.convert(conf.slow_query_log_timeout_in_ms, MILLISECONDS);
}
/**
* @return the minimum configured {read, write, range, truncate, misc} timeout
*/
- public static long getMinRpcTimeout()
+ public static long getMinRpcTimeout(TimeUnit unit)
{
- return Longs.min(getRpcTimeout(),
- getReadRpcTimeout(),
- getRangeRpcTimeout(),
- getWriteRpcTimeout(),
- getCounterWriteRpcTimeout(),
- getTruncateRpcTimeout());
+ return Longs.min(getRpcTimeout(unit),
+ getReadRpcTimeout(unit),
+ getRangeRpcTimeout(unit),
+ getWriteRpcTimeout(unit),
+ getCounterWriteRpcTimeout(unit),
+ getTruncateRpcTimeout(unit));
}
- public static long getPingTimeout()
+ public static long getPingTimeout(TimeUnit unit)
{
- return TimeUnit.SECONDS.toMillis(getBlockForPeersTimeoutInSeconds());
+ return unit.convert(getBlockForPeersTimeoutInSeconds(), TimeUnit.SECONDS);
}
public static double getPhiConvictThreshold()
@@ -1833,14 +1888,44 @@ public static boolean getRpcKeepAlive()
return conf.rpc_keepalive;
}
- public static int getInternodeSendBufferSize()
+ public static int getInternodeSocketSendBufferSizeInBytes()
+ {
+ return conf.internode_socket_send_buffer_size_in_bytes;
+ }
+
+ public static int getInternodeSocketReceiveBufferSizeInBytes()
+ {
+ return conf.internode_socket_receive_buffer_size_in_bytes;
+ }
+
+ public static int getInternodeApplicationSendQueueCapacityInBytes()
+ {
+ return conf.internode_application_send_queue_capacity_in_bytes;
+ }
+
+ public static int getInternodeApplicationSendQueueReserveEndpointCapacityInBytes()
{
- return conf.internode_send_buff_size_in_bytes;
+ return conf.internode_application_send_queue_reserve_endpoint_capacity_in_bytes;
}
- public static int getInternodeRecvBufferSize()
+ public static int getInternodeApplicationSendQueueReserveGlobalCapacityInBytes()
{
- return conf.internode_recv_buff_size_in_bytes;
+ return conf.internode_application_send_queue_reserve_global_capacity_in_bytes;
+ }
+
+ public static int getInternodeApplicationReceiveQueueCapacityInBytes()
+ {
+ return conf.internode_application_receive_queue_capacity_in_bytes;
+ }
+
+ public static int getInternodeApplicationReceiveQueueReserveEndpointCapacityInBytes()
+ {
+ return conf.internode_application_receive_queue_reserve_endpoint_capacity_in_bytes;
+ }
+
+ public static int getInternodeApplicationReceiveQueueReserveGlobalCapacityInBytes()
+ {
+ return conf.internode_application_receive_queue_reserve_global_capacity_in_bytes;
}
public static int getInternodeTcpConnectTimeoutInMS()
@@ -1863,6 +1948,17 @@ public static void setInternodeTcpUserTimeoutInMS(int value)
conf.internode_tcp_user_timeout_in_ms = value;
}
+ public static int getInternodeMaxMessageSizeInBytes()
+ {
+ return conf.internode_max_message_size_in_bytes;
+ }
+
+ @VisibleForTesting
+ public static void setInternodeMaxMessageSizeInBytes(int value)
+ {
+ conf.internode_max_message_size_in_bytes = value;
+ }
+
public static boolean startNativeTransport()
{
return conf.start_native_transport;
@@ -1954,6 +2050,26 @@ public static void setCommitLogSyncGroupWindow(double windowMillis)
conf.commitlog_sync_group_window_in_ms = windowMillis;
}
+ public static long getNativeTransportMaxConcurrentRequestsInBytesPerIp()
+ {
+ return conf.native_transport_max_concurrent_requests_in_bytes_per_ip;
+ }
+
+ public static void setNativeTransportMaxConcurrentRequestsInBytesPerIp(long maxConcurrentRequestsInBytes)
+ {
+ conf.native_transport_max_concurrent_requests_in_bytes_per_ip = maxConcurrentRequestsInBytes;
+ }
+
+ public static long getNativeTransportMaxConcurrentRequestsInBytes()
+ {
+ return conf.native_transport_max_concurrent_requests_in_bytes;
+ }
+
+ public static void setNativeTransportMaxConcurrentRequestsInBytes(long maxConcurrentRequestsInBytes)
+ {
+ conf.native_transport_max_concurrent_requests_in_bytes = maxConcurrentRequestsInBytes;
+ }
+
public static int getCommitLogSyncPeriod()
{
return conf.commitlog_sync_period_in_ms;
@@ -2142,6 +2258,12 @@ public static EncryptionOptions getNativeProtocolEncryptionOptions()
return conf.client_encryption_options;
}
+ @VisibleForTesting
+ public static void updateNativeProtocolEncryptionOptions(Function update)
+ {
+ conf.client_encryption_options = update.apply(conf.client_encryption_options);
+ }
+
public static int getHintedHandoffThrottleInKB()
{
return conf.hinted_handoff_throttle_in_kb;
@@ -2485,41 +2607,6 @@ public static int getTracetypeQueryTTL()
return conf.tracetype_query_ttl;
}
- public static String getOtcCoalescingStrategy()
- {
- return conf.otc_coalescing_strategy;
- }
-
- public static void setOtcCoalescingStrategy(String strategy)
- {
- conf.otc_coalescing_strategy = strategy;
- }
-
- public static int getOtcCoalescingWindow()
- {
- return conf.otc_coalescing_window_us;
- }
-
- public static int getOtcCoalescingEnoughCoalescedMessages()
- {
- return conf.otc_coalescing_enough_coalesced_messages;
- }
-
- public static void setOtcCoalescingEnoughCoalescedMessages(int otc_coalescing_enough_coalesced_messages)
- {
- conf.otc_coalescing_enough_coalesced_messages = otc_coalescing_enough_coalesced_messages;
- }
-
- public static int getOtcBacklogExpirationInterval()
- {
- return conf.otc_backlog_expiration_interval_ms;
- }
-
- public static void setOtcBacklogExpirationInterval(int intervalInMillis)
- {
- conf.otc_backlog_expiration_interval_ms = intervalInMillis;
- }
-
public static int getWindowsTimerInterval()
{
return conf.windows_timer_interval;
@@ -2825,4 +2912,15 @@ public static boolean strictRuntimeChecks()
{
return strictRuntimeChecks;
}
+
+ public static boolean useOffheapMerkleTrees()
+ {
+ return conf.use_offheap_merkle_trees;
+ }
+
+ public static void useOffheapMerkleTrees(boolean value)
+ {
+ logger.info("Setting use_offheap_merkle_trees to {}", value);
+ conf.use_offheap_merkle_trees = value;
+ }
}
diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java
index 9524cec36ec0..0a33dcc66778 100644
--- a/src/java/org/apache/cassandra/config/EncryptionOptions.java
+++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java
@@ -17,30 +17,61 @@
*/
package org.apache.cassandra.config;
-import java.util.Arrays;
+import java.util.List;
import java.util.Objects;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+
public class EncryptionOptions
{
- public String keystore = "conf/.keystore";
- public String keystore_password = "cassandra";
- public String truststore = "conf/.truststore";
- public String truststore_password = "cassandra";
- public String[] cipher_suites = {};
- public String protocol = "TLS";
- public String algorithm = null;
- public String store_type = "JKS";
- public boolean require_client_auth = false;
- public boolean require_endpoint_verification = false;
- public boolean enabled = false;
- public boolean optional = false;
+ public final String keystore;
+ public final String keystore_password;
+ public final String truststore;
+ public final String truststore_password;
+ public final List cipher_suites;
+ public final String protocol;
+ public final String algorithm;
+ public final String store_type;
+ public final boolean require_client_auth;
+ public final boolean require_endpoint_verification;
+ public final boolean enabled;
+ public final boolean optional;
public EncryptionOptions()
- { }
+ {
+ keystore = "conf/.keystore";
+ keystore_password = "cassandra";
+ truststore = "conf/.truststore";
+ truststore_password = "cassandra";
+ cipher_suites = ImmutableList.of();
+ protocol = "TLS";
+ algorithm = null;
+ store_type = "JKS";
+ require_client_auth = false;
+ require_endpoint_verification = false;
+ enabled = false;
+ optional = false;
+ }
+
+ public EncryptionOptions(String keystore, String keystore_password, String truststore, String truststore_password, List cipher_suites, String protocol, String algorithm, String store_type, boolean require_client_auth, boolean require_endpoint_verification, boolean enabled, boolean optional)
+ {
+ this.keystore = keystore;
+ this.keystore_password = keystore_password;
+ this.truststore = truststore;
+ this.truststore_password = truststore_password;
+ this.cipher_suites = cipher_suites;
+ this.protocol = protocol;
+ this.algorithm = algorithm;
+ this.store_type = store_type;
+ this.require_client_auth = require_client_auth;
+ this.require_endpoint_verification = require_endpoint_verification;
+ this.enabled = enabled;
+ this.optional = optional;
+ }
- /**
- * Copy constructor
- */
public EncryptionOptions(EncryptionOptions options)
{
keystore = options.keystore;
@@ -57,6 +88,97 @@ public EncryptionOptions(EncryptionOptions options)
optional = options.optional;
}
+ public EncryptionOptions withKeyStore(String keystore)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withKeyStorePassword(String keystore_password)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withTrustStore(String truststore)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withTrustStorePassword(String truststore_password)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withCipherSuites(List cipher_suites)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withCipherSuites(String ... cipher_suites)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, ImmutableList.copyOf(cipher_suites),
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withProtocol(String protocol)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withAlgorithm(String algorithm)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withStoreType(String store_type)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withRequireClientAuth(boolean require_client_auth)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withRequireEndpointVerification(boolean require_endpoint_verification)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withEnabled(boolean enabled)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
+ public EncryptionOptions withOptional(boolean optional)
+ {
+ return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional);
+ }
+
/**
* The method is being mainly used to cache SslContexts therefore, we only consider
* fields that would make a difference when the TrustStore or KeyStore files are updated
@@ -81,7 +203,7 @@ public boolean equals(Object o)
Objects.equals(protocol, opt.protocol) &&
Objects.equals(algorithm, opt.algorithm) &&
Objects.equals(store_type, opt.store_type) &&
- Arrays.equals(cipher_suites, opt.cipher_suites);
+ Objects.equals(cipher_suites, opt.cipher_suites);
}
/**
@@ -101,7 +223,7 @@ public int hashCode()
result += 31 * (store_type == null ? 0 : store_type.hashCode());
result += 31 * Boolean.hashCode(enabled);
result += 31 * Boolean.hashCode(optional);
- result += 31 * Arrays.hashCode(cipher_suites);
+ result += 31 * (cipher_suites == null ? 0 : cipher_suites.hashCode());
result += 31 * Boolean.hashCode(require_client_auth);
result += 31 * Boolean.hashCode(require_endpoint_verification);
return result;
@@ -114,20 +236,156 @@ public enum InternodeEncryption
all, none, dc, rack
}
- public InternodeEncryption internode_encryption = InternodeEncryption.none;
- public boolean enable_legacy_ssl_storage_port = false;
+ public final InternodeEncryption internode_encryption;
+ public final boolean enable_legacy_ssl_storage_port;
public ServerEncryptionOptions()
- { }
+ {
+ this.internode_encryption = InternodeEncryption.none;
+ this.enable_legacy_ssl_storage_port = false;
+ }
+ public ServerEncryptionOptions(String keystore, String keystore_password, String truststore, String truststore_password, List cipher_suites, String protocol, String algorithm, String store_type, boolean require_client_auth, boolean require_endpoint_verification, boolean enabled, boolean optional, InternodeEncryption internode_encryption, boolean enable_legacy_ssl_storage_port)
+ {
+ super(keystore, keystore_password, truststore, truststore_password, cipher_suites, protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional);
+ this.internode_encryption = internode_encryption;
+ this.enable_legacy_ssl_storage_port = enable_legacy_ssl_storage_port;
+ }
- /**
- * Copy constructor
- */
public ServerEncryptionOptions(ServerEncryptionOptions options)
{
super(options);
- internode_encryption = options.internode_encryption;
- enable_legacy_ssl_storage_port = options.enable_legacy_ssl_storage_port;
+ this.internode_encryption = options.internode_encryption;
+ this.enable_legacy_ssl_storage_port = options.enable_legacy_ssl_storage_port;
}
+
+ public boolean shouldEncrypt(InetAddressAndPort endpoint)
+ {
+ IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+ switch (internode_encryption)
+ {
+ case none:
+ return false; // if nothing needs to be encrypted then return immediately.
+ case all:
+ break;
+ case dc:
+ if (snitch.getDatacenter(endpoint).equals(snitch.getLocalDatacenter()))
+ return false;
+ break;
+ case rack:
+ // for rack then check if the DC's are the same.
+ if (snitch.getRack(endpoint).equals(snitch.getLocalRack())
+ && snitch.getDatacenter(endpoint).equals(snitch.getLocalDatacenter()))
+ return false;
+ break;
+ }
+ return true;
+ }
+
+
+ public ServerEncryptionOptions withKeyStore(String keystore)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withKeyStorePassword(String keystore_password)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withTrustStore(String truststore)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withTrustStorePassword(String truststore_password)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withCipherSuites(List cipher_suites)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withCipherSuites(String ... cipher_suites)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, ImmutableList.copyOf(cipher_suites),
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withProtocol(String protocol)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withAlgorithm(String algorithm)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withStoreType(String store_type)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withRequireClientAuth(boolean require_client_auth)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withRequireEndpointVerification(boolean require_endpoint_verification)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withEnabled(boolean enabled)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withOptional(boolean optional)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withInternodeEncryption(InternodeEncryption internode_encryption)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
+ public ServerEncryptionOptions withLegacySslStoragePort(boolean enable_legacy_ssl_storage_port)
+ {
+ return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+ protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+ enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+ }
+
}
}
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
index c348cc4f8ace..6410e67a70e6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
@@ -166,7 +166,7 @@ private void addColumn(KeyspaceMetadata keyspace,
{
// After #8099, not safe to re-add columns of incompatible types - until *maybe* deser logic with dropped
// columns is pushed deeper down the line. The latter would still be problematic in cases of schema races.
- if (!droppedColumn.type.isValueCompatibleWith(type))
+ if (!type.isValueCompatibleWith(droppedColumn.type))
{
throw ire("Cannot re-add previously dropped column '%s' of type %s, incompatible with previous type %s",
name,
diff --git a/src/java/org/apache/cassandra/db/AbstractCompactionController.java b/src/java/org/apache/cassandra/db/AbstractCompactionController.java
new file mode 100644
index 000000000000..99193f8626a5
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/AbstractCompactionController.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.db;
+
+import java.util.function.LongPredicate;
+
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.schema.CompactionParams;
+
+/**
+ * AbstractCompactionController allows custom implementations of the CompactionController for use in tooling, without being tied to the SSTableReader and local filesystem
+ */
+public abstract class AbstractCompactionController implements AutoCloseable
+{
+ public final ColumnFamilyStore cfs;
+ public final int gcBefore;
+ public final CompactionParams.TombstoneOption tombstoneOption;
+
+ public AbstractCompactionController(final ColumnFamilyStore cfs, final int gcBefore, CompactionParams.TombstoneOption tombstoneOption)
+ {
+ assert cfs != null;
+ this.cfs = cfs;
+ this.gcBefore = gcBefore;
+ this.tombstoneOption = tombstoneOption;
+ }
+
+ public abstract boolean compactingRepaired();
+
+ public String getKeyspace()
+ {
+ return cfs.keyspace.getName();
+ }
+
+ public String getColumnFamily()
+ {
+ return cfs.name;
+ }
+
+ public Iterable shadowSources(DecoratedKey key, boolean tombstoneOnly)
+ {
+ return null;
+ }
+
+ public abstract LongPredicate getPurgeEvaluator(DecoratedKey key);
+}
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c09b88477f43..f20d74c1d702 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -87,6 +87,9 @@
import org.json.simple.JSONArray;
import org.json.simple.JSONObject;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination;
+import static org.apache.cassandra.utils.ExecutorUtils.shutdown;
import static org.apache.cassandra.utils.Throwables.maybeFail;
public class ColumnFamilyStore implements ColumnFamilyStoreMBean
@@ -217,31 +220,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
private volatile boolean neverPurgeTombstones = false;
- public static void shutdownFlushExecutor() throws InterruptedException
- {
- flushExecutor.shutdown();
- flushExecutor.awaitTermination(60, TimeUnit.SECONDS);
- }
-
-
public static void shutdownPostFlushExecutor() throws InterruptedException
{
postFlushExecutor.shutdown();
postFlushExecutor.awaitTermination(60, TimeUnit.SECONDS);
}
- public static void shutdownReclaimExecutor() throws InterruptedException
- {
- reclaimExecutor.shutdown();
- reclaimExecutor.awaitTermination(60, TimeUnit.SECONDS);
- }
-
- public static void shutdownPerDiskFlushExecutors() throws InterruptedException
+ public static void shutdownExecutorsAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
{
- for (ExecutorService executorService : perDiskflushExecutors)
- executorService.shutdown();
- for (ExecutorService executorService : perDiskflushExecutors)
- executorService.awaitTermination(60, TimeUnit.SECONDS);
+ List executors = new ArrayList<>(perDiskflushExecutors.length + 3);
+ Collections.addAll(executors, reclaimExecutor, postFlushExecutor, flushExecutor);
+ Collections.addAll(executors, perDiskflushExecutors);
+ ExecutorUtils.shutdownAndWait(timeout, unit, executors);
}
public void reload()
@@ -295,7 +285,7 @@ protected void runMayThrow()
else
{
// we'll be rescheduled by the constructor of the Memtable.
- forceFlush();
+ forceFlushToSSTable();
}
}
}
@@ -401,8 +391,8 @@ public ColumnFamilyStore(Keyspace keyspace,
viewManager = keyspace.viewManager.forTable(metadata.id);
metric = new TableMetrics(this);
fileIndexGenerator.set(generation);
- sampleReadLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getReadRpcTimeout() / 2);
- additionalWriteLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getWriteRpcTimeout() / 2);
+ sampleReadLatencyNanos = DatabaseDescriptor.getReadRpcTimeout(NANOSECONDS) / 2;
+ additionalWriteLatencyNanos = DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS) / 2;
logger.info("Initializing {}.{}", keyspace.getName(), name);
@@ -868,7 +858,7 @@ private void logFlush()
* @return a Future yielding the commit log position that can be guaranteed to have been successfully written
* to sstables for this table once the future completes
*/
- public ListenableFuture forceFlush()
+ public ListenableFuture forceFlushToSSTable()
{
synchronized (data)
{
@@ -887,7 +877,7 @@ public ListenableFuture forceFlush()
* @return a Future yielding the commit log position that can be guaranteed to have been successfully written
* to sstables for this table once the future completes
*/
- public ListenableFuture> forceFlush(CommitLogPosition flushIfDirtyBefore)
+ public ListenableFuture> forceFlushToSSTable(CommitLogPosition flushIfDirtyBefore)
{
// we don't loop through the remaining memtables since here we only care about commit log dirtiness
// and this does not vary between a table and its table-backed indexes
@@ -914,9 +904,9 @@ private ListenableFuture waitForFlushes()
return task;
}
- public CommitLogPosition forceBlockingFlush()
+ public CommitLogPosition forceBlockingFlushToSSTable()
{
- return FBUtilities.waitOnFuture(forceFlush());
+ return FBUtilities.waitOnFuture(forceFlushToSSTable());
}
/**
@@ -1896,7 +1886,7 @@ public Set snapshot(String snapshotName, Predicate
{
if (!skipFlush)
{
- forceBlockingFlush();
+ forceBlockingFlushToSSTable();
}
return snapshotWithoutFlush(snapshotName, predicate, ephemeral);
}
@@ -2120,7 +2110,7 @@ public void truncateBlocking()
if (keyspace.getMetadata().params.durableWrites || DatabaseDescriptor.isAutoSnapshot())
{
- replayAfter = forceBlockingFlush();
+ replayAfter = forceBlockingFlushToSSTable();
viewManager.forceBlockingFlush();
}
else
@@ -2390,14 +2380,14 @@ private void validateCompactionThresholds(int minThreshold, int maxThreshold)
// End JMX get/set.
- public int getMeanColumns()
+ public int getMeanEstimatedCellPerPartitionCount()
{
long sum = 0;
long count = 0;
for (SSTableReader sstable : getSSTables(SSTableSet.CANONICAL))
{
- long n = sstable.getEstimatedColumnCount().count();
- sum += sstable.getEstimatedColumnCount().mean() * n;
+ long n = sstable.getEstimatedCellPerPartitionCount().count();
+ sum += sstable.getEstimatedCellPerPartitionCount().mean() * n;
count += n;
}
return count > 0 ? (int) (sum / count) : 0;
@@ -2416,6 +2406,19 @@ public double getMeanPartitionSize()
return count > 0 ? sum * 1.0 / count : 0;
}
+ public int getMeanRowCount()
+ {
+ long totalRows = 0;
+ long totalPartitions = 0;
+ for (SSTableReader sstable : getSSTables(SSTableSet.CANONICAL))
+ {
+ totalPartitions += sstable.getEstimatedPartitionSize().count();
+ totalRows += sstable.getTotalRows();
+ }
+
+ return totalPartitions > 0 ? (int) (totalRows / totalPartitions) : 0;
+ }
+
public long estimateKeys()
{
long n = 0;
@@ -2556,7 +2559,7 @@ public double getDroppableTombstoneRatio()
for (SSTableReader sstable : getSSTables(SSTableSet.LIVE))
{
allDroppable += sstable.getDroppableTombstonesBefore(localTime - metadata().params.gcGraceSeconds);
- allColumns += sstable.getEstimatedColumnCount().mean() * sstable.getEstimatedColumnCount().count();
+ allColumns += sstable.getEstimatedCellPerPartitionCount().mean() * sstable.getEstimatedCellPerPartitionCount().count();
}
return allColumns > 0 ? allDroppable / allColumns : 0;
}
diff --git a/src/java/org/apache/cassandra/db/Columns.java b/src/java/org/apache/cassandra/db/Columns.java
index 50817399ca93..bf9e17492dda 100644
--- a/src/java/org/apache/cassandra/db/Columns.java
+++ b/src/java/org/apache/cassandra/db/Columns.java
@@ -28,6 +28,7 @@
import com.google.common.hash.Hasher;
import net.nicoulaj.compilecommand.annotations.DontInline;
+import org.apache.cassandra.exceptions.UnknownColumnException;
import org.apache.cassandra.schema.ColumnMetadata;
import org.apache.cassandra.schema.TableMetadata;
import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -454,7 +455,7 @@ public Columns deserialize(DataInputPlus in, TableMetadata metadata) throws IOEx
// deserialization. The column will be ignore later on anyway.
column = metadata.getDroppedColumn(name);
if (column == null)
- throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
+ throw new UnknownColumnException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
}
builder.add(column);
}
diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java
index d04ddd8b909f..bb10a6a7a956 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -24,7 +24,6 @@
import com.google.common.base.Function;
import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
import com.google.common.collect.PeekingIterator;
@@ -39,14 +38,14 @@
import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.schema.TableId;
import org.apache.cassandra.service.CacheService;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.btree.BTreeSet;
+import static java.util.concurrent.TimeUnit.*;
+
public class CounterMutation implements IMutation
{
public static final CounterMutationSerializer serializer = new CounterMutationSerializer();
@@ -92,11 +91,6 @@ public ConsistencyLevel consistency()
return consistency;
}
- public MessageOut makeMutationMessage()
- {
- return new MessageOut<>(MessagingService.Verb.COUNTER_MUTATION, this, serializer);
- }
-
/**
* Applies the counter mutation, returns the result Mutation (for replication to other nodes).
*
@@ -146,10 +140,10 @@ private void grabCounterLocks(Keyspace keyspace, List locks) throws WriteT
for (Lock lock : LOCKS.bulkGet(getCounterLockKeys()))
{
- long timeout = TimeUnit.MILLISECONDS.toNanos(getTimeout()) - (System.nanoTime() - startTime);
+ long timeout = getTimeout(NANOSECONDS) - (System.nanoTime() - startTime);
try
{
- if (!lock.tryLock(timeout, TimeUnit.NANOSECONDS))
+ if (!lock.tryLock(timeout, NANOSECONDS))
throw new WriteTimeoutException(WriteType.COUNTER, consistency(), 0, consistency().blockFor(keyspace));
locks.add(lock);
}
@@ -309,9 +303,9 @@ private void updateForRow(PeekingIterator markIter,
}
}
- public long getTimeout()
+ public long getTimeout(TimeUnit unit)
{
- return DatabaseDescriptor.getCounterWriteRpcTimeout();
+ return DatabaseDescriptor.getCounterWriteRpcTimeout(unit);
}
@Override
diff --git a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
index c946ea595fef..a30ce665beba 100644
--- a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
@@ -22,16 +22,17 @@
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.utils.FBUtilities;
public class CounterMutationVerbHandler implements IVerbHandler
{
+ public static final CounterMutationVerbHandler instance = new CounterMutationVerbHandler();
+
private static final Logger logger = LoggerFactory.getLogger(CounterMutationVerbHandler.class);
- public void doVerb(final MessageIn message, final int id)
+ public void doVerb(final Message message)
{
long queryStartNanoTime = System.nanoTime();
final CounterMutation cm = message.payload;
@@ -45,12 +46,9 @@ public void doVerb(final MessageIn message, final int id)
// will not be called if the request timeout, but this is ok
// because the coordinator of the counter mutation will timeout on
// it's own in that case.
- StorageProxy.applyCounterMutationOnLeader(cm, localDataCenter, new Runnable()
- {
- public void run()
- {
- MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from);
- }
- }, queryStartNanoTime);
+ StorageProxy.applyCounterMutationOnLeader(cm,
+ localDataCenter,
+ () -> MessagingService.instance().send(message.emptyResponse(), message.from()),
+ queryStartNanoTime);
}
}
diff --git a/src/java/org/apache/cassandra/db/IMutation.java b/src/java/org/apache/cassandra/db/IMutation.java
index 9eaf19b4922f..1710cfd12379 100644
--- a/src/java/org/apache/cassandra/db/IMutation.java
+++ b/src/java/org/apache/cassandra/db/IMutation.java
@@ -18,6 +18,7 @@
package org.apache.cassandra.db;
import java.util.Collection;
+import java.util.concurrent.TimeUnit;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.schema.TableId;
@@ -28,7 +29,7 @@ public interface IMutation
public String getKeyspaceName();
public Collection getTableIds();
public DecoratedKey key();
- public long getTimeout();
+ public long getTimeout(TimeUnit unit);
public String toString(boolean shallow);
public Collection getPartitionUpdates();
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index bc382eef82b8..fa1f7c87debb 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -56,6 +56,9 @@
import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.concurrent.OpOrder;
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+
/**
* It represents a Keyspace.
*/
@@ -395,7 +398,7 @@ public void dropCf(TableId tableId)
// disassociate a cfs from this keyspace instance.
private void unloadCf(ColumnFamilyStore cfs)
{
- cfs.forceBlockingFlush();
+ cfs.forceBlockingFlushToSSTable();
cfs.invalidate();
}
@@ -544,7 +547,7 @@ private CompletableFuture> applyInternal(final Mutation mutation,
if (lock == null)
{
//throw WTE only if request is droppable
- if (isDroppable && (System.currentTimeMillis() - mutation.createdAt) > DatabaseDescriptor.getWriteRpcTimeout())
+ if (isDroppable && (approxTime.isAfter(mutation.approxCreatedAtNanos + DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS))))
{
for (int j = 0; j < i; j++)
locks[j].unlock();
@@ -605,7 +608,7 @@ else if (isDeferrable)
if (isDroppable)
{
for(TableId tableId : tableIds)
- columnFamilyStores.get(tableId).metric.viewLockAcquireTime.update(acquireTime, TimeUnit.MILLISECONDS);
+ columnFamilyStores.get(tableId).metric.viewLockAcquireTime.update(acquireTime, MILLISECONDS);
}
}
int nowInSec = FBUtilities.nowInSeconds();
@@ -671,7 +674,7 @@ public List> flush()
{
List> futures = new ArrayList<>(columnFamilyStores.size());
for (ColumnFamilyStore cfs : columnFamilyStores.values())
- futures.add(cfs.forceFlush());
+ futures.add(cfs.forceFlushToSSTable());
return futures;
}
diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java
index 6195fe4c87a5..22c4ed83766f 100644
--- a/src/java/org/apache/cassandra/db/Mutation.java
+++ b/src/java/org/apache/cassandra/db/Mutation.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.util.*;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import com.google.common.collect.ImmutableCollection;
@@ -32,13 +33,13 @@
import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.schema.TableId;
import org.apache.cassandra.schema.TableMetadata;
import org.apache.cassandra.utils.ByteBufferUtil;
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+
public class Mutation implements IMutation
{
public static final MutationSerializer serializer = new MutationSerializer();
@@ -52,7 +53,7 @@ public class Mutation implements IMutation
private final ImmutableMap modifications;
// Time at which this mutation or the builder that built it was instantiated
- final long createdAt;
+ final long approxCreatedAtNanos;
// keep track of when mutation has started waiting for a MV partition lock
final AtomicLong viewLockAcquireStart = new AtomicLong(0);
@@ -60,10 +61,10 @@ public class Mutation implements IMutation
public Mutation(PartitionUpdate update)
{
- this(update.metadata().keyspace, update.partitionKey(), ImmutableMap.of(update.metadata().id, update), System.currentTimeMillis());
+ this(update.metadata().keyspace, update.partitionKey(), ImmutableMap.of(update.metadata().id, update), approxTime.now());
}
- public Mutation(String keyspaceName, DecoratedKey key, ImmutableMap modifications, long createdAt)
+ public Mutation(String keyspaceName, DecoratedKey key, ImmutableMap modifications, long approxCreatedAtNanos)
{
this.keyspaceName = keyspaceName;
this.key = key;
@@ -73,7 +74,7 @@ public Mutation(String keyspaceName, DecoratedKey key, ImmutableMap tableIds)
@@ -90,7 +91,7 @@ public Mutation without(Set tableIds)
}
}
- return new Mutation(keyspaceName, key, builder.build(), createdAt);
+ return new Mutation(keyspaceName, key, builder.build(), approxCreatedAtNanos);
}
public Mutation without(TableId tableId)
@@ -177,7 +178,7 @@ public static Mutation merge(List mutations)
modifications.put(table, updates.size() == 1 ? updates.get(0) : PartitionUpdate.merge(updates));
updates.clear();
}
- return new Mutation(ks, key, modifications.build(), System.currentTimeMillis());
+ return new Mutation(ks, key, modifications.build(), approxTime.now());
}
public CompletableFuture> applyFuture()
@@ -210,19 +211,9 @@ public void applyUnsafe()
apply(false);
}
- public MessageOut createMessage()
- {
- return createMessage(MessagingService.Verb.MUTATION);
- }
-
- public MessageOut createMessage(MessagingService.Verb verb)
- {
- return new MessageOut<>(verb, this, serializer);
- }
-
- public long getTimeout()
+ public long getTimeout(TimeUnit unit)
{
- return DatabaseDescriptor.getWriteRpcTimeout();
+ return DatabaseDescriptor.getWriteRpcTimeout(unit);
}
public int smallestGCGS()
@@ -363,7 +354,7 @@ public Mutation deserialize(DataInputPlus in, int version, SerializationHelper.F
update = PartitionUpdate.serializer.deserialize(in, version, flag);
modifications.put(update.metadata().id, update);
}
- return new Mutation(update.metadata().keyspace, dk, modifications.build(), System.currentTimeMillis());
+ return new Mutation(update.metadata().keyspace, dk, modifications.build(), approxTime.now());
}
public Mutation deserialize(DataInputPlus in, int version) throws IOException
@@ -389,7 +380,7 @@ public static class PartitionUpdateCollector
private final ImmutableMap.Builder modifications = new ImmutableMap.Builder<>();
private final String keyspaceName;
private final DecoratedKey key;
- private final long createdAt = System.currentTimeMillis();
+ private final long approxCreatedAtNanos = approxTime.now();
private boolean empty = true;
public PartitionUpdateCollector(String keyspaceName, DecoratedKey key)
@@ -425,7 +416,7 @@ public boolean isEmpty()
public Mutation build()
{
- return new Mutation(keyspaceName, key, modifications.build(), createdAt);
+ return new Mutation(keyspaceName, key, modifications.build(), approxCreatedAtNanos);
}
}
}
diff --git a/src/java/org/apache/cassandra/db/MutationVerbHandler.java b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
index 9660f658dd35..bcb9cc7aaee3 100644
--- a/src/java/org/apache/cassandra/db/MutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
@@ -17,8 +17,6 @@
*/
package org.apache.cassandra.db;
-import java.util.Iterator;
-
import org.apache.cassandra.exceptions.WriteTimeoutException;
import org.apache.cassandra.locator.InetAddressAndPort;
import org.apache.cassandra.net.*;
@@ -26,10 +24,12 @@
public class MutationVerbHandler implements IVerbHandler
{
- private void reply(int id, InetAddressAndPort replyTo)
+ public static final MutationVerbHandler instance = new MutationVerbHandler();
+
+ private void respond(Message> respondTo, InetAddressAndPort respondToAddress)
{
- Tracing.trace("Enqueuing response to {}", replyTo);
- MessagingService.instance().sendReply(WriteResponse.createMessage(), id, replyTo);
+ Tracing.trace("Enqueuing response to {}", respondToAddress);
+ MessagingService.instance().send(respondTo.emptyResponse(), respondToAddress);
}
private void failed()
@@ -37,27 +37,25 @@ private void failed()
Tracing.trace("Payload application resulted in WriteTimeout, not replying");
}
- public void doVerb(MessageIn message, int id)
+ public void doVerb(Message message)
{
// Check if there were any forwarding headers in this message
- InetAddressAndPort from = (InetAddressAndPort)message.parameters.get(ParameterType.FORWARD_FROM);
- InetAddressAndPort replyTo;
+ InetAddressAndPort from = message.respondTo();
+ InetAddressAndPort respondToAddress;
if (from == null)
{
- replyTo = message.from;
- ForwardToContainer forwardTo = (ForwardToContainer)message.parameters.get(ParameterType.FORWARD_TO);
- if (forwardTo != null)
- forwardToLocalNodes(message.payload, message.verb, forwardTo, message.from);
+ respondToAddress = message.from();
+ ForwardingInfo forwardTo = message.forwardTo();
+ if (forwardTo != null) forwardToLocalNodes(message, forwardTo);
}
else
{
-
- replyTo = from;
+ respondToAddress = from;
}
try
{
- message.payload.applyFuture().thenAccept(o -> reply(id, replyTo)).exceptionally(wto -> {
+ message.payload.applyFuture().thenAccept(o -> respond(message, respondToAddress)).exceptionally(wto -> {
failed();
return null;
});
@@ -68,17 +66,21 @@ public void doVerb(MessageIn message, int id)
}
}
- private static void forwardToLocalNodes(Mutation mutation, MessagingService.Verb verb, ForwardToContainer forwardTo, InetAddressAndPort from)
+ private static void forwardToLocalNodes(Message originalMessage, ForwardingInfo forwardTo)
{
- // tell the recipients who to send their ack to
- MessageOut message = new MessageOut<>(verb, mutation, Mutation.serializer).withParameter(ParameterType.FORWARD_FROM, from);
- Iterator iterator = forwardTo.targets.iterator();
- // Send a message to each of the addresses on our Forward List
- for (int i = 0; i < forwardTo.targets.size(); i++)
+ Message.Builder builder =
+ Message.builder(originalMessage)
+ .withParam(ParamType.RESPOND_TO, originalMessage.from())
+ .withoutParam(ParamType.FORWARD_TO);
+
+ boolean useSameMessageID = forwardTo.useSameMessageID();
+ // reuse the same Message if all ids are identical (as they will be for 4.0+ node originated messages)
+ Message message = useSameMessageID ? builder.build() : null;
+
+ forwardTo.forEach((id, target) ->
{
- InetAddressAndPort address = iterator.next();
- Tracing.trace("Enqueuing forwarded write to {}", address);
- MessagingService.instance().sendOneWay(message, forwardTo.messageIds[i], address);
- }
+ Tracing.trace("Enqueuing forwarded write to {}", target);
+ MessagingService.instance().send(useSameMessageID ? message : builder.withId(id).build(), target);
+ });
}
}
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index b5f6fb534214..2145389607a6 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -18,9 +18,12 @@
package org.apache.cassandra.db;
import java.io.IOException;
+import java.util.concurrent.TimeUnit;
import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.net.MessageFlag;
+import org.apache.cassandra.net.Verb;
import org.apache.cassandra.schema.TableMetadata;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.filter.*;
@@ -38,8 +41,7 @@
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.metrics.TableMetrics;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.Message;
import org.apache.cassandra.schema.IndexMetadata;
import org.apache.cassandra.service.ClientState;
import org.apache.cassandra.service.StorageProxy;
@@ -233,9 +235,9 @@ public PartitionRangeReadCommand withUpdatedLimitsAndDataRange(DataLimits newLim
indexMetadata());
}
- public long getTimeout()
+ public long getTimeout(TimeUnit unit)
{
- return DatabaseDescriptor.getRangeRpcTimeout();
+ return DatabaseDescriptor.getRangeRpcTimeout(unit);
}
public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException
@@ -345,9 +347,10 @@ public BaseRowIterator applyToPartition(BaseRowIterator iter)
return Transformation.apply(iter, new CacheFilter());
}
- public MessageOut createMessage()
+ @Override
+ public Verb verb()
{
- return new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, serializer);
+ return Verb.RANGE_REQ;
}
protected void appendCQLWhereClause(StringBuilder sb)
@@ -414,6 +417,11 @@ public boolean isLimitedToOnePartition()
&& dataRange.startKey().equals(dataRange.stopKey());
}
+ public boolean isRangeRequest()
+ {
+ return true;
+ }
+
private static class Deserializer extends SelectionDeserializer
{
public ReadCommand deserialize(DataInputPlus in,
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 32b91ad8905e..68ce2eacbf9b 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.*;
+import java.util.concurrent.TimeUnit;
import java.util.function.BiFunction;
import java.util.function.LongPredicate;
@@ -35,7 +36,9 @@
import org.apache.cassandra.config.*;
import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.monitoring.ApproximateTime;
+import org.apache.cassandra.net.MessageFlag;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.utils.ApproximateTime;
import org.apache.cassandra.db.partitions.*;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.db.transform.RTBoundCloser;
@@ -52,9 +55,8 @@
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.locator.Replica;
-import org.apache.cassandra.locator.ReplicaCollection;
import org.apache.cassandra.metrics.TableMetrics;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
import org.apache.cassandra.schema.IndexMetadata;
import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.schema.SchemaConstants;
@@ -69,6 +71,7 @@
import static com.google.common.collect.Iterables.any;
import static com.google.common.collect.Iterables.filter;
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
/**
* General interface for storage-engine read commands (common to both range and
@@ -164,6 +167,8 @@ protected ReadCommand(Kind kind,
public abstract boolean isLimitedToOnePartition();
+ public abstract boolean isRangeRequest();
+
/**
* Creates a new ReadCommand instance with new limits.
*
@@ -177,7 +182,7 @@ protected ReadCommand(Kind kind,
*
* @return the configured timeout for this command.
*/
- public abstract long getTimeout();
+ public abstract long getTimeout(TimeUnit unit);
/**
* Whether this query is a digest one or not.
@@ -628,14 +633,15 @@ protected Row applyToRow(Row row)
private boolean maybeAbort()
{
/**
- * The value returned by ApproximateTime.currentTimeMillis() is updated only every
- * {@link ApproximateTime.CHECK_INTERVAL_MS}, by default 10 millis. Since MonitorableImpl
- * relies on ApproximateTime, we don't need to check unless the approximate time has elapsed.
+ * TODO: this is not a great way to abort early; why not expressly limit checks to 10ms intervals?
+ * The value returned by approxTime.now() is updated only every
+ * {@link org.apache.cassandra.utils.MonotonicClock.SampledClock.CHECK_INTERVAL_MS}, by default 2 millis. Since MonitorableImpl
+ * relies on approxTime, we don't need to check unless the approximate time has elapsed.
*/
- if (lastChecked == ApproximateTime.currentTimeMillis())
+ if (lastChecked == approxTime.now())
return false;
- lastChecked = ApproximateTime.currentTimeMillis();
+ lastChecked = approxTime.now();
if (isAborted())
{
@@ -661,7 +667,14 @@ protected UnfilteredPartitionIterator withStateTracking(UnfilteredPartitionItera
/**
* Creates a message for this command.
*/
- public abstract MessageOut createMessage();
+ public Message createMessage(boolean trackRepairedData)
+ {
+ return trackRepairedData
+ ? Message.outWithFlags(verb(), this, MessageFlag.CALL_BACK_ON_FAILURE, MessageFlag.TRACK_REPAIRED_DATA)
+ : Message.outWithFlag (verb(), this, MessageFlag.CALL_BACK_ON_FAILURE);
+ }
+
+ public abstract Verb verb();
protected abstract void appendCQLWhereClause(StringBuilder sb);
diff --git a/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java b/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java
index e39e8a855205..2c28ed9d4b8f 100644
--- a/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java
@@ -20,29 +20,26 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.locator.Replica;
import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.ParameterType;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.tracing.Tracing;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
public class ReadCommandVerbHandler implements IVerbHandler
{
- private static final Logger logger = LoggerFactory.getLogger(ReadCommandVerbHandler.class);
+ public static final ReadCommandVerbHandler instance = new ReadCommandVerbHandler();
- protected IVersionedSerializer serializer()
- {
- return ReadResponse.serializer;
- }
+ private static final Logger logger = LoggerFactory.getLogger(ReadCommandVerbHandler.class);
- public void doVerb(MessageIn message, int id)
+ public void doVerb(Message message)
{
if (StorageService.instance.isBootstrapMode())
{
@@ -52,9 +49,10 @@ public void doVerb(MessageIn message, int id)
ReadCommand command = message.payload;
validateTransientStatus(message);
- command.setMonitoringTime(message.constructionTime, message.isCrossNode(), message.getTimeout(), message.getSlowQueryTimeout());
+ long timeout = message.expiresAtNanos() - message.createdAtNanos();
+ command.setMonitoringTime(message.createdAtNanos(), message.isCrossNode(), timeout, DatabaseDescriptor.getSlowQueryTimeout(NANOSECONDS));
- if (message.parameters.containsKey(ParameterType.TRACK_REPAIRED_DATA))
+ if (message.trackRepairedData())
command.trackRepairedStatus();
ReadResponse response;
@@ -66,17 +64,17 @@ public void doVerb(MessageIn message, int id)
if (!command.complete())
{
- Tracing.trace("Discarding partial response to {} (timed out)", message.from);
- MessagingService.instance().incrementDroppedMessages(message, message.getLifetimeInMS());
+ Tracing.trace("Discarding partial response to {} (timed out)", message.from());
+ MessagingService.instance().metrics.recordDroppedMessage(message, message.elapsedSinceCreated(NANOSECONDS), NANOSECONDS);
return;
}
- Tracing.trace("Enqueuing response to {}", message.from);
- MessageOut reply = new MessageOut<>(MessagingService.Verb.REQUEST_RESPONSE, response, serializer());
- MessagingService.instance().sendReply(reply, id, message.from);
+ Tracing.trace("Enqueuing response to {}", message.from());
+ Message reply = message.responseWith(response);
+ MessagingService.instance().send(reply, message.from());
}
- private void validateTransientStatus(MessageIn message)
+ private void validateTransientStatus(Message message)
{
ReadCommand command = message.payload;
Token token;
@@ -93,14 +91,14 @@ private void validateTransientStatus(MessageIn message)
if (replica == null)
{
logger.warn("Received a read request from {} for a range that is not owned by the current replica {}.",
- message.from,
+ message.from(),
command);
return;
}
if (!command.acceptsTransient() && replica.isTransient())
{
- MessagingService.instance().incrementDroppedMessages(message, message.getLifetimeInMS());
+ MessagingService.instance().metrics.recordDroppedMessage(message, message.elapsedSinceCreated(NANOSECONDS), NANOSECONDS);
throw new InvalidRequestException(String.format("Attempted to serve %s data request from %s node in %s",
command.acceptsTransient() ? "transient" : "full",
replica.isTransient() ? "transient" : "full",
diff --git a/src/java/org/apache/cassandra/db/ReadExecutionController.java b/src/java/org/apache/cassandra/db/ReadExecutionController.java
index 29b6fa7b0484..73ddad8022d8 100644
--- a/src/java/org/apache/cassandra/db/ReadExecutionController.java
+++ b/src/java/org/apache/cassandra/db/ReadExecutionController.java
@@ -21,9 +21,11 @@
import org.apache.cassandra.index.Index;
import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.MonotonicClock;
import org.apache.cassandra.utils.concurrent.OpOrder;
+import static org.apache.cassandra.utils.MonotonicClock.preciseTime;
+
public class ReadExecutionController implements AutoCloseable
{
private static final long NO_SAMPLING = Long.MIN_VALUE;
@@ -36,7 +38,7 @@ public class ReadExecutionController implements AutoCloseable
private final ReadExecutionController indexController;
private final WriteContext writeContext;
private final ReadCommand command;
- static Clock clock = Clock.instance;
+ static MonotonicClock clock = preciseTime;
private final long createdAtNanos; // Only used while sampling
@@ -93,7 +95,7 @@ static ReadExecutionController forCommand(ReadCommand command)
ColumnFamilyStore baseCfs = Keyspace.openAndGetStore(command.metadata());
ColumnFamilyStore indexCfs = maybeGetIndexCfs(baseCfs, command);
- long createdAtNanos = baseCfs.metric.topLocalReadQueryTime.isEnabled() ? clock.nanoTime() : NO_SAMPLING;
+ long createdAtNanos = baseCfs.metric.topLocalReadQueryTime.isEnabled() ? clock.now() : NO_SAMPLING;
if (indexCfs == null)
return new ReadExecutionController(command, baseCfs.readOrdering.start(), baseCfs.metadata(), null, null, createdAtNanos);
@@ -172,7 +174,7 @@ public void close()
private void addSample()
{
String cql = command.toCQLString();
- int timeMicros = (int) Math.min(TimeUnit.NANOSECONDS.toMicros(clock.nanoTime() - createdAtNanos), Integer.MAX_VALUE);
+ int timeMicros = (int) Math.min(TimeUnit.NANOSECONDS.toMicros(clock.now() - createdAtNanos), Integer.MAX_VALUE);
ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(baseMetadata.id);
if (cfs != null)
cfs.metric.topLocalReadQueryTime.addSample(cql, timeMicros);
diff --git a/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java b/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java
index 2e499e7935c5..903b3d43bdf3 100644
--- a/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java
@@ -18,14 +18,16 @@
package org.apache.cassandra.db;
import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
import org.apache.cassandra.net.MessagingService;
public class ReadRepairVerbHandler implements IVerbHandler
{
- public void doVerb(MessageIn message, int id)
+ public static final ReadRepairVerbHandler instance = new ReadRepairVerbHandler();
+
+ public void doVerb(Message message)
{
message.payload.apply();
- MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from);
+ MessagingService.instance().send(message.emptyResponse(), message.from());
}
}
diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java
index deadf68785b1..2e5211c087e8 100644
--- a/src/java/org/apache/cassandra/db/SerializationHeader.java
+++ b/src/java/org/apache/cassandra/db/SerializationHeader.java
@@ -27,6 +27,7 @@
import org.apache.cassandra.db.marshal.TypeParser;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.exceptions.UnknownColumnException;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.Version;
import org.apache.cassandra.io.sstable.metadata.IMetadataComponentSerializer;
@@ -292,7 +293,7 @@ public MetadataType getType()
return MetadataType.HEADER;
}
- public SerializationHeader toHeader(TableMetadata metadata)
+ public SerializationHeader toHeader(TableMetadata metadata) throws UnknownColumnException
{
Map> typeMap = new HashMap<>(staticColumns.size() + regularColumns.size());
@@ -320,7 +321,7 @@ public SerializationHeader toHeader(TableMetadata metadata)
// deserialization. The column will be ignore later on anyway.
column = metadata.getDroppedColumn(name, isStatic);
if (column == null)
- throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
+ throw new UnknownColumnException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
}
builder.add(column);
}
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index aec1a54cedb7..8c983aa164f8 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -20,10 +20,9 @@
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.*;
+import java.util.concurrent.TimeUnit;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import org.apache.cassandra.cache.IRowCacheEntry;
@@ -43,12 +42,11 @@
import org.apache.cassandra.io.sstable.format.SSTableReadsListener;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.locator.Replica;
-import org.apache.cassandra.locator.ReplicaCollection;
import org.apache.cassandra.metrics.TableMetrics;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessageFlag;
import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.ParameterType;
+import org.apache.cassandra.net.Verb;
import org.apache.cassandra.schema.ColumnMetadata;
import org.apache.cassandra.schema.IndexMetadata;
import org.apache.cassandra.schema.TableMetadata;
@@ -364,9 +362,9 @@ public ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key)
return clusteringIndexFilter;
}
- public long getTimeout()
+ public long getTimeout(TimeUnit unit)
{
- return DatabaseDescriptor.getReadRpcTimeout();
+ return DatabaseDescriptor.getReadRpcTimeout(unit);
}
@Override
@@ -1040,9 +1038,10 @@ public String toString()
nowInSec());
}
- public MessageOut createMessage()
+ @Override
+ public Verb verb()
{
- return new MessageOut<>(MessagingService.Verb.READ, this, serializer);
+ return Verb.READ_REQ;
}
protected void appendCQLWhereClause(StringBuilder sb)
@@ -1078,6 +1077,11 @@ public boolean isLimitedToOnePartition()
return true;
}
+ public boolean isRangeRequest()
+ {
+ return false;
+ }
+
/**
* Groups multiple single partition read commands.
*/
diff --git a/src/java/org/apache/cassandra/db/SnapshotCommand.java b/src/java/org/apache/cassandra/db/SnapshotCommand.java
index eb6f67a028e7..484db2fd7178 100644
--- a/src/java/org/apache/cassandra/db/SnapshotCommand.java
+++ b/src/java/org/apache/cassandra/db/SnapshotCommand.java
@@ -22,8 +22,8 @@
import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.Verb;
public class SnapshotCommand
{
@@ -42,11 +42,6 @@ public SnapshotCommand(String keyspace, String columnFamily, String snapshotName
this.clear_snapshot = clearSnapshot;
}
- public MessageOut createMessage()
- {
- return new MessageOut(MessagingService.Verb.SNAPSHOT, this, serializer);
- }
-
@Override
public String toString()
{
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index d48f84fc3f0e..4c586f407452 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -688,16 +688,17 @@ public static synchronized void updateTokens(InetAddressAndPort ep, Collection serializer = new Serializer();
+
+ public final String keyspace;
+ public final String table;
+
+ public TruncateRequest(String keyspace, String table)
+ {
+ this.keyspace = keyspace;
+ this.table = table;
+ }
+
+ @Override
+ public String toString()
+ {
+ return String.format("TruncateRequest(keyspace='%s', table='%s')'", keyspace, table);
+ }
+
+ private static class Serializer implements IVersionedSerializer
+ {
+ public void serialize(TruncateRequest request, DataOutputPlus out, int version) throws IOException
+ {
+ out.writeUTF(request.keyspace);
+ out.writeUTF(request.table);
+ }
+
+ public TruncateRequest deserialize(DataInputPlus in, int version) throws IOException
+ {
+ String keyspace = in.readUTF();
+ String table = in.readUTF();
+ return new TruncateRequest(keyspace, table);
+ }
+
+ public long serializedSize(TruncateRequest request, int version)
+ {
+ return TypeSizes.sizeof(request.keyspace) + TypeSizes.sizeof(request.table);
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/db/TruncateResponse.java b/src/java/org/apache/cassandra/db/TruncateResponse.java
index af4ed8f2f929..822c9ccea30f 100644
--- a/src/java/org/apache/cassandra/db/TruncateResponse.java
+++ b/src/java/org/apache/cassandra/db/TruncateResponse.java
@@ -22,8 +22,6 @@
import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
/**
* This message is sent back the truncate operation and basically specifies if
@@ -44,11 +42,6 @@ public TruncateResponse(String keyspace, String columnFamily, boolean success)
this.success = success;
}
- public MessageOut createMessage()
- {
- return new MessageOut(MessagingService.Verb.REQUEST_RESPONSE, this, serializer);
- }
-
public static class TruncateResponseSerializer implements IVersionedSerializer
{
public void serialize(TruncateResponse tr, DataOutputPlus out, int version) throws IOException
diff --git a/src/java/org/apache/cassandra/db/TruncateVerbHandler.java b/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
index c2fac6561c87..c605d1f20e78 100644
--- a/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
@@ -22,21 +22,23 @@
import org.apache.cassandra.io.FSError;
import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.tracing.Tracing;
-public class TruncateVerbHandler implements IVerbHandler
+public class TruncateVerbHandler implements IVerbHandler
{
+ public static final TruncateVerbHandler instance = new TruncateVerbHandler();
+
private static final Logger logger = LoggerFactory.getLogger(TruncateVerbHandler.class);
- public void doVerb(MessageIn message, int id)
+ public void doVerb(Message message)
{
- Truncation t = message.payload;
- Tracing.trace("Applying truncation of {}.{}", t.keyspace, t.columnFamily);
+ TruncateRequest t = message.payload;
+ Tracing.trace("Applying truncation of {}.{}", t.keyspace, t.table);
try
{
- ColumnFamilyStore cfs = Keyspace.open(t.keyspace).getColumnFamilyStore(t.columnFamily);
+ ColumnFamilyStore cfs = Keyspace.open(t.keyspace).getColumnFamilyStore(t.table);
cfs.truncateBlocking();
}
catch (Exception e)
@@ -47,16 +49,16 @@ public void doVerb(MessageIn message, int id)
if (FSError.findNested(e) != null)
throw FSError.findNested(e);
}
- Tracing.trace("Enqueuing response to truncate operation to {}", message.from);
+ Tracing.trace("Enqueuing response to truncate operation to {}", message.from());
- TruncateResponse response = new TruncateResponse(t.keyspace, t.columnFamily, true);
- logger.trace("{} applied. Enqueuing response to {}@{} ", t, id, message.from );
- MessagingService.instance().sendReply(response.createMessage(), id, message.from);
+ TruncateResponse response = new TruncateResponse(t.keyspace, t.table, true);
+ logger.trace("{} applied. Enqueuing response to {}@{} ", t, message.id(), message.from());
+ MessagingService.instance().send(message.responseWith(response), message.from());
}
- private static void respondError(Truncation t, MessageIn truncateRequestMessage)
+ private static void respondError(TruncateRequest t, Message truncateRequestMessage)
{
- TruncateResponse response = new TruncateResponse(t.keyspace, t.columnFamily, false);
- MessagingService.instance().sendOneWay(response.createMessage(), truncateRequestMessage.from);
+ TruncateResponse response = new TruncateResponse(t.keyspace, t.table, false);
+ MessagingService.instance().send(truncateRequestMessage.responseWith(response), truncateRequestMessage.from());
}
}
diff --git a/src/java/org/apache/cassandra/db/Truncation.java b/src/java/org/apache/cassandra/db/Truncation.java
deleted file mode 100644
index 39a2ec6b98a4..000000000000
--- a/src/java/org/apache/cassandra/db/Truncation.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-
-/**
- * A truncate operation descriptor
- */
-public class Truncation
-{
- public static final IVersionedSerializer serializer = new TruncationSerializer();
-
- public final String keyspace;
- public final String columnFamily;
-
- public Truncation(String keyspace, String columnFamily)
- {
- this.keyspace = keyspace;
- this.columnFamily = columnFamily;
- }
-
- public MessageOut createMessage()
- {
- return new MessageOut(MessagingService.Verb.TRUNCATE, this, serializer);
- }
-
- public String toString()
- {
- return "Truncation(" + "keyspace='" + keyspace + '\'' + ", cf='" + columnFamily + "\')";
- }
-}
-
-class TruncationSerializer implements IVersionedSerializer
-{
- public void serialize(Truncation t, DataOutputPlus out, int version) throws IOException
- {
- out.writeUTF(t.keyspace);
- out.writeUTF(t.columnFamily);
- }
-
- public Truncation deserialize(DataInputPlus in, int version) throws IOException
- {
- String keyspace = in.readUTF();
- String columnFamily = in.readUTF();
- return new Truncation(keyspace, columnFamily);
- }
-
- public long serializedSize(Truncation truncation, int version)
- {
- return TypeSizes.sizeof(truncation.keyspace) + TypeSizes.sizeof(truncation.columnFamily);
- }
-}
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 60a6f70534e4..1e1030cf3dfe 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -108,7 +108,7 @@ protected ReusablePartitionData createBuffer(int blocksCount)
// FIXME: so far we only keep stats on cells, so to get a rough estimate on the number of rows,
// we divide by the number of regular columns the table has. We should fix once we collect the
// stats on rows
- int estimatedRowsPerPartition = (int)(sstable.getEstimatedColumnCount().percentile(0.75) / columnCount);
+ int estimatedRowsPerPartition = (int)(sstable.getEstimatedCellPerPartitionCount().percentile(0.75) / columnCount);
estimatedRowCount = Math.max(estimatedRowsPerPartition / blocksCount, 1);
}
catch (IllegalStateException e)
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
index b7ab70592c06..45df4eb46733 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
@@ -30,7 +30,7 @@
import org.apache.cassandra.concurrent.NamedThreadFactory;
import org.apache.cassandra.config.Config;
import org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
-import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.MonotonicClock;
import org.apache.cassandra.utils.NoSpamLogger;
import org.apache.cassandra.utils.concurrent.WaitQueue;
@@ -133,21 +133,21 @@ void start()
throw new IllegalArgumentException(String.format("Commit log flush interval must be positive: %fms",
syncIntervalNanos * 1e-6));
shutdown = false;
- Runnable runnable = new SyncRunnable(new Clock());
+ Runnable runnable = new SyncRunnable(MonotonicClock.preciseTime);
thread = NamedThreadFactory.createThread(runnable, name);
thread.start();
}
class SyncRunnable implements Runnable
{
- private final Clock clock;
+ private final MonotonicClock clock;
private long firstLagAt = 0;
private long totalSyncDuration = 0; // total time spent syncing since firstLagAt
private long syncExceededIntervalBy = 0; // time that syncs exceeded pollInterval since firstLagAt
private int lagCount = 0;
private int syncCount = 0;
- SyncRunnable(Clock clock)
+ SyncRunnable(MonotonicClock clock)
{
this.clock = clock;
}
@@ -169,7 +169,7 @@ boolean sync()
try
{
// sync and signal
- long pollStarted = clock.nanoTime();
+ long pollStarted = clock.now();
boolean flushToDisk = lastSyncedAt + syncIntervalNanos <= pollStarted || shutdownRequested || syncRequested;
if (flushToDisk)
{
@@ -186,7 +186,7 @@ boolean sync()
commitLog.sync(false);
}
- long now = clock.nanoTime();
+ long now = clock.now();
if (flushToDisk)
maybeLogFlushLag(pollStarted, now);
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index 9d2a3695ca85..1b3fe2e3ad97 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -67,7 +67,7 @@ public class CommitLog implements CommitLogMBean
// empty segments when writing large records
final long MAX_MUTATION_SIZE = DatabaseDescriptor.getMaxMutationSize();
- final public AbstractCommitLogSegmentManager segmentManager;
+ final public CommitLogSegmentManager segmentManager;
public final CommitLogArchiver archiver;
final CommitLogMetrics metrics;
@@ -108,9 +108,7 @@ private static CommitLog construct()
throw new IllegalArgumentException("Unknown commitlog service type: " + DatabaseDescriptor.getCommitLogSync());
}
- segmentManager = DatabaseDescriptor.isCDCEnabled()
- ? new CommitLogSegmentManagerCDC(this, DatabaseDescriptor.getCommitLogLocation())
- : new CommitLogSegmentManagerStandard(this, DatabaseDescriptor.getCommitLogLocation());
+ segmentManager = new CommitLogSegmentManager(this, DatabaseDescriptor.getCommitLogLocation());
// register metrics
metrics.attach(executor, segmentManager);
@@ -288,7 +286,7 @@ public CommitLogPosition add(Mutation mutation) throws CDCWriteException
}
catch (IOException e)
{
- throw new FSWriteError(e, segmentManager.allocatingFrom().getPath());
+ throw new FSWriteError(e, segmentManager.getActiveSegment().getPath());
}
}
@@ -308,7 +306,7 @@ public void discardCompletedSegments(final TableId id, final CommitLogPosition l
// flushed CF as clean, until we reach the segment file containing the CommitLogPosition passed
// in the arguments. Any segments that become unused after they are marked clean will be
// recycled or discarded.
- for (Iterator iter = segmentManager.getActiveSegments().iterator(); iter.hasNext();)
+ for (Iterator iter = segmentManager.getSegmentsForUnflushedTables().iterator(); iter.hasNext();)
{
CommitLogSegment segment = iter.next();
segment.markClean(id, lowerBound, upperBound);
@@ -364,7 +362,7 @@ public String getRestorePrecision()
public List getActiveSegmentNames()
{
- Collection segments = segmentManager.getActiveSegments();
+ Collection segments = segmentManager.getSegmentsForUnflushedTables();
List segmentNames = new ArrayList<>(segments.size());
for (CommitLogSegment seg : segments)
segmentNames.add(seg.getName());
@@ -380,7 +378,7 @@ public List getArchivingSegmentNames()
public long getActiveContentSize()
{
long size = 0;
- for (CommitLogSegment seg : segmentManager.getActiveSegments())
+ for (CommitLogSegment seg : segmentManager.getSegmentsForUnflushedTables())
size += seg.contentSize();
return size;
}
@@ -395,13 +393,14 @@ public long getActiveOnDiskSize()
public Map getActiveSegmentCompressionRatios()
{
Map segmentRatios = new TreeMap<>();
- for (CommitLogSegment seg : segmentManager.getActiveSegments())
+ for (CommitLogSegment seg : segmentManager.getSegmentsForUnflushedTables())
segmentRatios.put(seg.getName(), 1.0 * seg.onDiskSize() / seg.contentSize());
return segmentRatios;
}
/**
* Shuts down the threads used by the commit log, blocking until completion.
+ * TODO this should accept a timeout, and throw TimeoutException
*/
public void shutdownBlocking() throws InterruptedException
{
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
index 700f12a242ca..2d04126ff3d5 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
@@ -62,7 +62,7 @@ public class CommitLogDescriptor
public static final int VERSION_40 = 7;
/**
- * Increment this number if there is a changes in the commit log disc layout or MessagingVersion changes.
+ * Increment this number if there is a changes in the commit log disk layout or MessagingVersion changes.
* Note: make sure to handle {@link #getMessagingVersion()}
*/
@VisibleForTesting
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
index 078bb5304d41..7fb63bb27724 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
@@ -24,6 +24,8 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.zip.CRC32;
+import javax.annotation.Nonnull;
+
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
@@ -43,42 +45,96 @@
import org.apache.cassandra.io.util.RebufferingInputStream;
import org.apache.cassandra.schema.TableId;
import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Pair;
import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;
+/**
+ * The CommitLogReader presents an idempotentinterface for legacy CommitLogSegment reads. The logic to read and
+ * parse a CommitLogSegment is housed here, however this depends upon {@link ResumableCommitLogReader} for any non-trivial
+ * or resumable coordination of reads.
+ */
public class CommitLogReader
{
private static final Logger logger = LoggerFactory.getLogger(CommitLogReader.class);
private static final int LEGACY_END_OF_SEGMENT_MARKER = 0;
+ /** Used to indicate we want to read to the end of a commit log segment during logic for resumable reading */
+ static final int READ_TO_END_OF_FILE = Integer.MAX_VALUE;
+
@VisibleForTesting
- public static final int ALL_MUTATIONS = -1;
+ static final int ALL_MUTATIONS = -1;
private final CRC32 checksum;
private final Map invalidMutations;
+ @Nonnull
private byte[] buffer;
- public CommitLogReader()
+ CommitLogReader()
{
checksum = new CRC32();
invalidMutations = new HashMap<>();
buffer = new byte[4096];
}
- public Set> getInvalidMutations()
+ Set> getInvalidMutations()
{
return invalidMutations.entrySet();
}
- /**
- * Reads all passed in files with no minimum, no start, and no mutation limit.
- */
+ /** Reads all passed in files with no minimum, no start, and no mutation limit. */
public void readAllFiles(CommitLogReadHandler handler, File[] files) throws IOException
{
readAllFiles(handler, files, CommitLogPosition.NONE);
}
+ /** Reads all passed in files with minPosition, no start, and no mutation limit. */
+ public void readAllFiles(CommitLogReadHandler handler, File[] files, CommitLogPosition minPosition) throws IOException
+ {
+ List filteredLogs = filterCommitLogFiles(files);
+ int i = 0;
+ for (File file: filteredLogs)
+ {
+ i++;
+ readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, i == filteredLogs.size());
+ }
+ }
+
+ /** Read a CommitLogSegment fully, no restrictions */
+ void readCommitLogSegment(CommitLogReadHandler handler, File file, boolean tolerateTruncation) throws IOException
+ {
+ readCommitLogSegment(handler, file, CommitLogPosition.NONE, ALL_MUTATIONS, tolerateTruncation);
+ }
+
+ /** Read passed in file fully, up to mutationLimit count */
+ @VisibleForTesting
+ void readCommitLogSegment(CommitLogReadHandler handler, File file, int mutationLimit, boolean tolerateTruncation) throws IOException
+ {
+ readCommitLogSegment(handler, file, CommitLogPosition.NONE, mutationLimit, tolerateTruncation);
+ }
+
+ /** Read all mutations from passed in file from minPosition in the logical CommitLog */
+ void readCommitLogSegment(CommitLogReadHandler handler, File file, CommitLogPosition minPosition, boolean tolerateTruncation) throws IOException
+ {
+ readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, tolerateTruncation);
+ }
+
+ void readCommitLogSegment(CommitLogReadHandler handler,
+ File file,
+ CommitLogPosition minPosition,
+ int mutationLimit,
+ boolean tolerateTruncation) throws IOException
+ {
+ // TODO: Consider removing the need to create a resumable reader here and instead simply build and use the needed components.
+ // This would require a change to internalRead since that assumes a ResumableReader for convenience.
+ try(ResumableCommitLogReader resumableReader = new ResumableCommitLogReader(file, handler, minPosition, mutationLimit, tolerateTruncation))
+ {
+ resumableReader.readToCompletion();
+ }
+ }
+
+ /** Confirms whether the passed in file is one we should read or skip based on whether it's empty and passes crc */
private static boolean shouldSkip(File file) throws IOException, ConfigurationException
{
try(RandomAccessReader reader = RandomAccessReader.open(file))
@@ -90,6 +146,7 @@ private static boolean shouldSkip(File file) throws IOException, ConfigurationEx
}
}
+ /** Filters list of passed in CommitLogSegments based on shouldSkip logic, specifically whether files are empty and pass crc. */
static List filterCommitLogFiles(File[] toFilter)
{
List filtered = new ArrayList<>(toFilter.length);
@@ -117,163 +174,126 @@ static List filterCommitLogFiles(File[] toFilter)
}
/**
- * Reads all passed in files with minPosition, no start, and no mutation limit.
- */
- public void readAllFiles(CommitLogReadHandler handler, File[] files, CommitLogPosition minPosition) throws IOException
- {
- List filteredLogs = filterCommitLogFiles(files);
- int i = 0;
- for (File file: filteredLogs)
- {
- i++;
- readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, i == filteredLogs.size());
- }
- }
-
- /**
- * Reads passed in file fully
- */
- public void readCommitLogSegment(CommitLogReadHandler handler, File file, boolean tolerateTruncation) throws IOException
- {
- readCommitLogSegment(handler, file, CommitLogPosition.NONE, ALL_MUTATIONS, tolerateTruncation);
- }
-
- /**
- * Reads all mutations from passed in file from minPosition
- */
- public void readCommitLogSegment(CommitLogReadHandler handler, File file, CommitLogPosition minPosition, boolean tolerateTruncation) throws IOException
- {
- readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, tolerateTruncation);
- }
-
- /**
- * Reads passed in file fully, up to mutationLimit count
- */
- @VisibleForTesting
- public void readCommitLogSegment(CommitLogReadHandler handler, File file, int mutationLimit, boolean tolerateTruncation) throws IOException
- {
- readCommitLogSegment(handler, file, CommitLogPosition.NONE, mutationLimit, tolerateTruncation);
- }
-
- /**
- * Reads mutations from file, handing them off to handler
- * @param handler Handler that will take action based on deserialized Mutations
- * @param file CommitLogSegment file to read
- * @param minPosition Optional minimum CommitLogPosition - all segments with id larger or matching w/greater position will be read
- * @param mutationLimit Optional limit on # of mutations to replay. Local ALL_MUTATIONS serves as marker to play all.
- * @param tolerateTruncation Whether or not we should allow truncation of this file or throw if EOF found
+ * Reads and constructs the {@link CommitLogDescriptor} portion of a File.
*
- * @throws IOException
+ * @return Pair, Integer> An optional descriptor and serialized header size
*/
- public void readCommitLogSegment(CommitLogReadHandler handler,
- File file,
- CommitLogPosition minPosition,
- int mutationLimit,
- boolean tolerateTruncation) throws IOException
+ static Pair, Integer> readCommitLogDescriptor(CommitLogReadHandler handler,
+ File file,
+ boolean tolerateTruncation) throws IOException
{
// just transform from the file name (no reading of headers) to determine version
CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName());
+ long segmentIdFromFilename = desc.id;
+ int descriptorSize = -1;
- try(RandomAccessReader reader = RandomAccessReader.open(file))
+ try(RandomAccessReader rawSegmentReader = RandomAccessReader.open(file))
{
- final long segmentIdFromFilename = desc.id;
try
{
// The following call can either throw or legitimately return null. For either case, we need to check
// desc outside this block and set it to null in the exception case.
- desc = CommitLogDescriptor.readHeader(reader, DatabaseDescriptor.getEncryptionContext());
+ desc = CommitLogDescriptor.readHeader(rawSegmentReader, DatabaseDescriptor.getEncryptionContext());
}
catch (Exception e)
{
desc = null;
}
+
if (desc == null)
{
- // don't care about whether or not the handler thinks we can continue. We can't w/out descriptor.
- // whether or not we can continue depends on whether this is the last segment
- handler.handleUnrecoverableError(new CommitLogReadException(
+ // Don't care about whether or not the handler thinks we can continue. We can't w/out descriptor.
+ // Whether or not we can continue depends on whether this is the last segment
+ handler.handleUnrecoverableError(new CommitLogReadHandler.CommitLogReadException(
String.format("Could not read commit log descriptor in file %s", file),
- CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR,
+ CommitLogReadHandler.CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR,
tolerateTruncation));
- return;
+ return Pair.create(Optional.empty(), -1);
}
+ // Continuing if our file name and descriptor mismatch is optional.
if (segmentIdFromFilename != desc.id)
{
- if (handler.shouldSkipSegmentOnError(new CommitLogReadException(String.format(
- "Segment id mismatch (filename %d, descriptor %d) in file %s", segmentIdFromFilename, desc.id, file),
- CommitLogReadErrorReason.RECOVERABLE_DESCRIPTOR_ERROR,
- false)))
+ if (handler.shouldSkipSegmentOnError(new CommitLogReadHandler.CommitLogReadException(
+ String.format("Segment id mismatch (filename %d, descriptor %d) in file %s", segmentIdFromFilename, desc.id, file),
+ CommitLogReadHandler.CommitLogReadErrorReason.RECOVERABLE_DESCRIPTOR_ERROR,
+ false)))
{
- return;
+ return Pair.create(Optional.empty(), -1);
}
}
- if (shouldSkipSegmentId(file, desc, minPosition))
- return;
-
- CommitLogSegmentReader segmentReader;
- try
- {
- segmentReader = new CommitLogSegmentReader(handler, desc, reader, tolerateTruncation);
- }
- catch(Exception e)
- {
- handler.handleUnrecoverableError(new CommitLogReadException(
- String.format("Unable to create segment reader for commit log file: %s", e),
- CommitLogReadErrorReason.UNRECOVERABLE_UNKNOWN_ERROR,
- tolerateTruncation));
- return;
- }
-
- try
- {
- ReadStatusTracker statusTracker = new ReadStatusTracker(mutationLimit, tolerateTruncation);
- for (CommitLogSegmentReader.SyncSegment syncSegment : segmentReader)
- {
- // Only tolerate truncation if we allow in both global and segment
- statusTracker.tolerateErrorsInSection = tolerateTruncation & syncSegment.toleratesErrorsInSection;
-
- // Skip segments that are completely behind the desired minPosition
- if (desc.id == minPosition.segmentId && syncSegment.endPosition < minPosition.position)
- continue;
-
- statusTracker.errorContext = String.format("Next section at %d in %s", syncSegment.fileStartPosition, desc.fileName());
+ descriptorSize = (int)rawSegmentReader.getPosition();
+ }
+ return Pair.create(Optional.of(desc), descriptorSize);
+ }
- readSection(handler, syncSegment.input, minPosition, syncSegment.endPosition, statusTracker, desc);
- if (!statusTracker.shouldContinue())
- break;
- }
- }
- // Unfortunately AbstractIterator cannot throw a checked exception, so we check to see if a RuntimeException
- // is wrapping an IOException.
- catch (RuntimeException re)
- {
- if (re.getCause() instanceof IOException)
- throw (IOException) re.getCause();
- throw re;
- }
- logger.debug("Finished reading {}", file);
+ /**
+ * Opens a RandomAccessReader to a CommitLogSegment _and does not close it_. Closed out in {@link ResumableCommitLogReader#close}
+ */
+ static CommitLogSegmentReader getCommitLogSegmentReader(ResumableCommitLogReader parent) throws IOException
+ {
+ CommitLogSegmentReader result;
+ try
+ {
+ result = new CommitLogSegmentReader(parent);
}
+ catch(Exception e)
+ {
+ parent.readHandler.handleUnrecoverableError(new CommitLogReadHandler.CommitLogReadException(
+ String.format("Unable to create segment reader for commit log file: %s", e),
+ CommitLogReadHandler.CommitLogReadErrorReason.UNRECOVERABLE_UNKNOWN_ERROR,
+ parent.tolerateTruncation));
+ // Regardless of whether this is in the node context and we allow the node to continue to run, this reader is
+ // dead.
+ parent.close();
+ return null;
+ }
+ return result;
}
/**
- * Any segment with id >= minPosition.segmentId is a candidate for read.
+ * Iterates over {@link CommitLogSegmentReader.SyncSegment} until it hits offset limit or end of iterator, based
+ * on the resumable reader's sentinel.
*/
- private boolean shouldSkipSegmentId(File file, CommitLogDescriptor desc, CommitLogPosition minPosition)
+ void internalReadCommitLogSegment(ResumableCommitLogReader rr) throws IOException
{
- logger.debug("Reading {} (CL version {}, messaging version {}, compression {})",
- file.getPath(),
- desc.version,
- desc.getMessagingVersion(),
- desc.compression);
+ try
+ {
+ ReadStatusTracker statusTracker = new ReadStatusTracker(rr.mutationLimit, rr.tolerateTruncation);
+
+ int lastSegmentEnd = -1;
+
+ while (lastSegmentEnd < rr.offsetLimit && rr.activeIterator.hasNext())
+ {
+ CommitLogSegmentReader.SyncSegment syncSegment = rr.activeIterator.next();
+ // Back out if we're at the end of our current partially written CL segment.
+ if (syncSegment == CommitLogSegmentReader.RESUMABLE_SENTINEL)
+ break;
+
+ lastSegmentEnd = syncSegment.endPosition;
+
+ statusTracker.tolerateErrorsInSection = rr.tolerateTruncation & syncSegment.toleratesErrorsInSection;
- if (minPosition.segmentId > desc.id)
+ // Skip segments that are completely behind the desired minPosition
+ if (rr.descriptor.id == rr.minPosition.segmentId && syncSegment.endPosition < rr.minPosition.position)
+ continue;
+
+ statusTracker.errorContext = String.format("Next section at %d in %s", syncSegment.fileStartPosition, rr.descriptor.fileName());
+
+ readSection(rr.readHandler, syncSegment.input, rr.minPosition, syncSegment.endPosition, statusTracker, rr.descriptor);
+ if (!statusTracker.shouldContinue())
+ break;
+ }
+ }
+ // Unfortunately AbstractIterator cannot throw a checked exception, so we check to see if a RuntimeException
+ // is wrapping an IOException.
+ catch (RuntimeException | IOException re)
{
- logger.trace("Skipping read of fully-flushed {}", file);
- return true;
+ if (re.getCause() instanceof IOException)
+ throw (IOException) re.getCause();
+ throw re;
}
- return false;
}
/**
@@ -403,7 +423,7 @@ private void readSection(CommitLogReadHandler handler,
}
/**
- * Deserializes and passes a Mutation to the ICommitLogReadHandler requested
+ * Deserializes and passes a Mutation to the CommitLogReadHandler requested
*
* @param handler Handler that will take action based on deserialized Mutations
* @param inputBuffer raw byte array w/Mutation data
@@ -482,48 +502,51 @@ protected void readMutation(CommitLogReadHandler handler,
*/
private static class CommitLogFormat
{
- public static long calculateClaimedChecksum(FileDataInput input, int commitLogVersion) throws IOException
+ static long calculateClaimedChecksum(FileDataInput input, int commitLogVersion) throws IOException
{
return input.readInt() & 0xffffffffL;
}
- public static void updateChecksum(CRC32 checksum, int serializedSize, int commitLogVersion)
+ static void updateChecksum(CRC32 checksum, int serializedSize, int commitLogVersion)
{
updateChecksumInt(checksum, serializedSize);
}
- public static long calculateClaimedCRC32(FileDataInput input, int commitLogVersion) throws IOException
+ static long calculateClaimedCRC32(FileDataInput input, int commitLogVersion) throws IOException
{
return input.readInt() & 0xffffffffL;
}
}
+ /**
+ * Caches the state needed for decision-making on multiple CommitLog Read operations. Used internally in the CommitLogReader
+ */
private static class ReadStatusTracker
{
private int mutationsLeft;
- public String errorContext = "";
- public boolean tolerateErrorsInSection;
+ String errorContext = "";
+ boolean tolerateErrorsInSection;
private boolean error;
- public ReadStatusTracker(int mutationLimit, boolean tolerateErrorsInSection)
+ ReadStatusTracker(int mutationLimit, boolean tolerateErrorsInSection)
{
this.mutationsLeft = mutationLimit;
this.tolerateErrorsInSection = tolerateErrorsInSection;
}
- public void addProcessedMutation()
+ void addProcessedMutation()
{
if (mutationsLeft == ALL_MUTATIONS)
return;
--mutationsLeft;
}
- public boolean shouldContinue()
+ boolean shouldContinue()
{
return !error && (mutationsLeft != 0 || mutationsLeft == ALL_MUTATIONS);
}
- public void requestTermination()
+ void requestTermination()
{
error = true;
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index 2947222e4d93..f63499ab23c0 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -167,8 +167,8 @@ public void replayFiles(File[] clogs) throws IOException
*/
private void handleCDCReplayCompletion(File f) throws IOException
{
- // Can only reach this point if CDC is enabled, thus we have a CDCSegmentManager
- ((CommitLogSegmentManagerCDC)CommitLog.instance.segmentManager).addCDCSize(f.length());
+ // Can only reach this point if CDC is enabled, thus we have a CDC Allocator
+ ((CommitLogSegmentAllocatorCDC)CommitLog.instance.segmentManager.segmentAllocator).addCDCSize(f.length());
File dest = new File(DatabaseDescriptor.getCDCLogLocation(), f.getName());
@@ -217,7 +217,7 @@ public int blockForWrites()
// also flush batchlog incase of any MV updates
if (!flushingSystem)
- futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceFlush());
+ futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceFlushToSSTable());
FBUtilities.waitOnFutures(futures);
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index 5303de9da40e..793605b860a5 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -32,6 +32,9 @@
import com.google.common.annotations.VisibleForTesting;
import org.cliffc.high_scale_lib.NonBlockingHashMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import com.codahale.metrics.Timer;
import org.apache.cassandra.config.*;
import org.apache.cassandra.db.Mutation;
@@ -49,13 +52,15 @@
import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;
-/*
+/**
* A single commit log file on disk. Manages creation of the file and writing mutations to disk,
* as well as tracking the last mutation position of any "dirty" CFs covered by the segment file. Segment
* files are initially allocated to a fixed size and can grow to accomidate a larger value if necessary.
*/
public abstract class CommitLogSegment
{
+ static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentManager.class);
+
private final static long idBase;
private CDCState cdcState = CDCState.PERMITTED;
@@ -65,7 +70,7 @@ public enum CDCState
FORBIDDEN,
CONTAINS
}
- Object cdcStateLock = new Object();
+ final Object cdcStateLock = new Object();
private final static AtomicInteger nextId = new AtomicInteger(1);
private static long replayLimitId;
@@ -80,20 +85,21 @@ public enum CDCState
replayLimitId = idBase = Math.max(System.currentTimeMillis(), maxId + 1);
}
- // The commit log entry overhead in bytes (int: length + int: head checksum + int: tail checksum)
- public static final int ENTRY_OVERHEAD_SIZE = 4 + 4 + 4;
+ /** The commit log entry overhead in bytes (int: length + int: head checksum + int: tail checksum) */
+ static final int ENTRY_OVERHEAD_SIZE = 4 + 4 + 4;
- // The commit log (chained) sync marker/header size in bytes (int: length + int: checksum [segmentId, position])
+ /** The commit log (chained) sync marker/header size in bytes (int: length + int: checksum [segmentId, position]) */
static final int SYNC_MARKER_SIZE = 4 + 4;
- // The OpOrder used to order appends wrt sync
+ /** The OpOrder used to order appends wrt sync */
private final OpOrder appendOrder = new OpOrder();
private final AtomicInteger allocatePosition = new AtomicInteger();
- // Everything before this offset has been synced and written. The SYNC_MARKER_SIZE bytes after
- // each sync are reserved, and point forwards to the next such offset. The final
- // sync marker in a segment will be zeroed out, or point to a position too close to the EOF to fit a marker.
+ /** Everything before this offset has been synced and written. The SYNC_MARKER_SIZE bytes after
+ * each sync are reserved, and point forwards to the next such offset. The final
+ * sync marker in a segment will be zeroed out, or point to a position too close to the EOF to fit a marker.
+ */
@VisibleForTesting
volatile int lastSyncedOffset;
@@ -103,34 +109,36 @@ public enum CDCState
*/
private volatile int lastMarkerOffset;
- // The end position of the buffer. Initially set to its capacity and updated to point to the last written position
- // as the segment is being closed.
- // No need to be volatile as writes are protected by appendOrder barrier.
+ /** The end position of the buffer. Initially set to its capacity and updated to point to the last written position
+ * as the segment is being closed.
+ * No need to be volatile as writes are protected by appendOrder barrier.
+ */
private int endOfBuffer;
- // a signal for writers to wait on to confirm the log message they provided has been written to disk
+ /** a signal for writers to wait on to confirm the log message they provided has been written to disk */
private final WaitQueue syncComplete = new WaitQueue();
- // a map of Cf->dirty interval in this segment; if interval is not covered by the clean set, the log contains unflushed data
+ /** a map of Cf->dirty interval in this segment; if interval is not covered by the clean set, the log contains unflushed data */
private final NonBlockingHashMap tableDirty = new NonBlockingHashMap<>(1024);
- // a map of Cf->clean intervals; separate map from above to permit marking Cfs clean whilst the log is still in use
+ /** a map of Cf->clean intervals; separate map from above to permit marking Cfs clean whilst the log is still in use */
private final ConcurrentHashMap tableClean = new ConcurrentHashMap<>();
public final long id;
+ /** The CommitLogSegment log file on disk */
final File logFile;
final FileChannel channel;
final int fd;
- protected final AbstractCommitLogSegmentManager manager;
+ protected final CommitLogSegmentManager manager;
ByteBuffer buffer;
private volatile boolean headerWritten;
public final CommitLogDescriptor descriptor;
- static CommitLogSegment createSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
+ static CommitLogSegment createSegment(CommitLog commitLog, CommitLogSegmentManager manager)
{
Configuration config = commitLog.configuration;
CommitLogSegment segment = config.useEncryption() ? new EncryptedSegment(commitLog, manager)
@@ -160,7 +168,7 @@ static long getNextId()
/**
* Constructs a new segment file.
*/
- CommitLogSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
+ CommitLogSegment(CommitLog commitLog, CommitLogSegmentManager manager)
{
this.manager = manager;
@@ -265,10 +273,10 @@ private int allocate(int size)
}
}
- // ensures no more of this segment is writeable, by allocating any unused section at the end and marking it discarded
+ // Ensures no more of this segment is writeable, by allocating any unused section at the end and marking it discarded.
void discardUnusedTail()
{
- // We guard this with the OpOrdering instead of synchronised due to potential dead-lock with ACLSM.advanceAllocatingFrom()
+ // We guard this with the OpOrdering instead of synchronised due to potential dead-lock with CLSM.switchToNewSegment()
// Ensures endOfBuffer update is reflected in the buffer end position picked up by sync().
// This actually isn't strictly necessary, as currently all calls to discardUnusedTail are executed either by the thread
// running sync or within a mutation already protected by this OpOrdering, but to prevent future potential mistakes,
@@ -310,6 +318,8 @@ void waitForModifications()
* Update the chained markers in the commit log buffer and possibly force a disk flush for this segment file.
*
* @param flush true if the segment should flush to disk; else, false for just updating the chained markers.
+ * Named such to disambiguate whether we're looking to flush associated memtables to disk
+ * or just this CL segment.
*/
synchronized void sync(boolean flush)
{
@@ -363,7 +373,6 @@ synchronized void sync(boolean flush)
sectionEnd = nextMarker;
}
-
if (flush || close)
{
flush(startMarker, sectionEnd);
@@ -383,7 +392,7 @@ synchronized void sync(boolean flush)
* in shared / memory-mapped buffers reflects un-synced data so we need an external sentinel for clients to read to
* determine actual durable data persisted.
*/
- public static void writeCDCIndexFile(CommitLogDescriptor desc, int offset, boolean complete)
+ static void writeCDCIndexFile(CommitLogDescriptor desc, int offset, boolean complete)
{
try(FileWriter writer = new FileWriter(new File(DatabaseDescriptor.getCDCLogLocation(), desc.cdcIndexFileName())))
{
@@ -425,7 +434,7 @@ protected static void writeSyncMarker(long id, ByteBuffer buffer, int offset, in
abstract void flush(int startMarker, int nextMarker);
- public boolean isStillAllocating()
+ boolean hasRoom()
{
return allocatePosition.get() < endOfBuffer;
}
@@ -445,7 +454,7 @@ void discard(boolean deleteFile)
/**
* @return the current CommitLogPosition for this log segment
*/
- public CommitLogPosition getCurrentCommitLogPosition()
+ CommitLogPosition getCurrentCommitLogPosition()
{
return new CommitLogPosition(id, allocatePosition.get());
}
@@ -469,7 +478,7 @@ public String getName()
/**
* @return a File object representing the CDC directory and this file name for hard-linking
*/
- public File getCDCFile()
+ File getCDCFile()
{
return new File(DatabaseDescriptor.getCDCLogLocation(), logFile.getName());
}
@@ -477,7 +486,7 @@ public File getCDCFile()
/**
* @return a File object representing the CDC Index file holding the offset and completion status of this segment
*/
- public File getCDCIndexFile()
+ File getCDCIndexFile()
{
return new File(DatabaseDescriptor.getCDCLogLocation(), descriptor.cdcIndexFileName());
}
@@ -539,7 +548,7 @@ protected void internalClose()
}
}
- public static void coverInMap(ConcurrentMap map, K key, int value)
+ private static void coverInMap(ConcurrentMap map, K key, int value)
{
IntegerInterval i = map.get(key);
if (i == null)
@@ -561,7 +570,7 @@ public static void coverInMap(ConcurrentMap map, K key, i
* @param startPosition the start of the range that is clean
* @param endPosition the end of the range that is clean
*/
- public synchronized void markClean(TableId tableId, CommitLogPosition startPosition, CommitLogPosition endPosition)
+ synchronized void markClean(TableId tableId, CommitLogPosition startPosition, CommitLogPosition endPosition)
{
if (startPosition.segmentId > id || endPosition.segmentId < id)
return;
@@ -576,7 +585,7 @@ public synchronized void markClean(TableId tableId, CommitLogPosition startPosit
private void removeCleanFromDirty()
{
// if we're still allocating from this segment, don't touch anything since it can't be done thread-safely
- if (isStillAllocating())
+ if (hasRoom())
return;
Iterator> iter = tableClean.entrySet().iterator();
@@ -619,9 +628,9 @@ public synchronized Collection getDirtyTableIds()
*/
public synchronized boolean isUnused()
{
- // if room to allocate, we're still in use as the active allocatingFrom,
+ // if room to allocate, we're still in use as the active segment,
// so we don't want to race with updates to tableClean with removeCleanFromDirty
- if (isStillAllocating())
+ if (hasRoom())
return false;
removeCleanFromDirty();
@@ -640,7 +649,7 @@ public boolean contains(CommitLogPosition context)
}
// For debugging, not fast
- public String dirtyString()
+ String dirtyString()
{
StringBuilder sb = new StringBuilder();
for (TableId tableId : getDirtyTableIds())
@@ -656,7 +665,7 @@ public String dirtyString()
abstract public long onDiskSize();
- public long contentSize()
+ long contentSize()
{
return lastSyncedOffset;
}
@@ -677,7 +686,7 @@ public int compare(File f, File f2)
}
}
- public CDCState getCDCState()
+ CDCState getCDCState()
{
return cdcState;
}
@@ -686,7 +695,7 @@ public CDCState getCDCState()
* Change the current cdcState on this CommitLogSegment. There are some restrictions on state transitions and this
* method is idempotent.
*/
- public void setCDCState(CDCState newState)
+ void setCDCState(CDCState newState)
{
if (newState == cdcState)
return;
@@ -748,7 +757,7 @@ void awaitDiskSync(Timer waitingOnCommit)
/**
* Returns the position in the CommitLogSegment at the end of this allocation.
*/
- public CommitLogPosition getCommitLogPosition()
+ CommitLogPosition getCommitLogPosition()
{
return new CommitLogPosition(segment.id, buffer.limit());
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocator.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocator.java
new file mode 100644
index 000000000000..883b825be13e
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocator.java
@@ -0,0 +1,55 @@
+/*
+ * 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.commitlog;
+
+import java.io.File;
+
+import org.apache.cassandra.db.Mutation;
+
+public interface CommitLogSegmentAllocator
+{
+ void start();
+ void shutdown();
+
+ /**
+ * Indicates that a segment file has been flushed and is no longer needed. This can perform blocking disk
+ * operations so use with caution in critical path.
+ *
+ * @param segment segment to be discarded
+ * @param delete whether or not the segment is safe to be deleted.
+ */
+ void discard(CommitLogSegment segment, boolean delete);
+
+ /**
+ * Allocate a segment. This is always expected to succeed so should throw some form of exception on failure to
+ * allocate; if you can't allocate a CLS, you can no longer write and the node is in a bad state.
+ */
+ CommitLogSegment.Allocation allocate(Mutation mutation, int size);
+
+ /**
+ * Hook to allow segment managers to track state surrounding creation of new segments. This method is called
+ * on a separate segment management thread instead of the critical path so longer-running operations are acceptable.
+ */
+ CommitLogSegment createSegment();
+
+ /**
+ * When segments complete replay, the allocator has a hook to take action at that time.
+ */
+ void handleReplayedSegment(final File file);
+}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java
similarity index 72%
rename from src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
rename to src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java
index bdd4f74ae1e6..bd96a23d47b2 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java
@@ -39,28 +39,39 @@
import org.apache.cassandra.utils.DirectorySizeCalculator;
import org.apache.cassandra.utils.NoSpamLogger;
-public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager
+/**
+ * A CommitLogSegmentAllocator that respects the configured total allowable CDC space on disk. On allocation of a mutation
+ * checks if it's on a table tracked by CDC and, if so, either throws an exception if at CDC limit or flags that segment
+ * as containing a CDC mutation if it's a new one.
+ *
+ * This code path is only exercised if cdc is enabled on a node. We pay the duplication cost of having both CDC and non
+ * allocators in order to keep the old allocator code clean and separate from this allocator, as well as to not introduce
+ * unnecessary operations on the critical path for nodes / users where they have no interest in CDC. May be worth considering
+ * unifying in the future should the perf implications of this be shown to be negligible, though the hard linking and
+ * size tracking is somewhat distasteful to have floating around on nodes where cdc is not in use (which we assume to be
+ * the majority).
+ */
+public class CommitLogSegmentAllocatorCDC implements CommitLogSegmentAllocator
{
- static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentManagerCDC.class);
+ static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentAllocatorCDC.class);
private final CDCSizeTracker cdcSizeTracker;
+ private final CommitLogSegmentManager segmentManager;
- public CommitLogSegmentManagerCDC(final CommitLog commitLog, String storageDirectory)
+ CommitLogSegmentAllocatorCDC(CommitLogSegmentManager segmentManager)
{
- super(commitLog, storageDirectory);
- cdcSizeTracker = new CDCSizeTracker(this, new File(DatabaseDescriptor.getCDCLogLocation()));
+ this.segmentManager = segmentManager;
+ cdcSizeTracker = new CDCSizeTracker(segmentManager, new File(DatabaseDescriptor.getCDCLogLocation()));
}
- @Override
- void start()
+ public void start()
{
cdcSizeTracker.start();
- super.start();
}
public void discard(CommitLogSegment segment, boolean delete)
{
segment.close();
- addSize(-segment.onDiskSize());
+ segmentManager.addSize(-segment.onDiskSize());
cdcSizeTracker.processDiscardedSegment(segment);
@@ -82,12 +93,11 @@ public void discard(CommitLogSegment segment, boolean delete)
}
/**
- * Initiates the shutdown process for the management thread. Also stops the cdc on-disk size calculator executor.
+ * Stops the thread pool for CDC on disk size tracking.
*/
public void shutdown()
{
cdcSizeTracker.shutdown();
- super.shutdown();
}
/**
@@ -99,20 +109,23 @@ public void shutdown()
* @return the created Allocation object
* @throws CDCWriteException If segment disallows CDC mutations, we throw
*/
- @Override
public CommitLogSegment.Allocation allocate(Mutation mutation, int size) throws CDCWriteException
{
- CommitLogSegment segment = allocatingFrom();
- CommitLogSegment.Allocation alloc;
-
+ CommitLogSegment segment = segmentManager.getActiveSegment();
throwIfForbidden(mutation, segment);
- while ( null == (alloc = segment.allocate(mutation, size)) )
+
+ CommitLogSegment.Allocation alloc = segment.allocate(mutation, size);
+ // If we failed to allocate in the segment, prompt for a switch to a new segment and loop on re-attempt. This
+ // is expected to succeed or throw, since CommitLog allocation working is central to how a node operates.
+ while (alloc == null)
{
// Failed to allocate, so move to a new segment with enough room if possible.
- advanceAllocatingFrom(segment);
- segment = allocatingFrom();
+ segmentManager.switchToNewSegment(segment);
+ segment = segmentManager.getActiveSegment();
+ // New segment, so confirm whether or not CDC mutations are allowed on this.
throwIfForbidden(mutation, segment);
+ alloc = segment.allocate(mutation, size);
}
if (mutation.trackedByCDC())
@@ -143,7 +156,7 @@ private void throwIfForbidden(Mutation mutation, CommitLogSegment segment) throw
*/
public CommitLogSegment createSegment()
{
- CommitLogSegment segment = CommitLogSegment.createSegment(commitLog, this);
+ CommitLogSegment segment = CommitLogSegment.createSegment(segmentManager.commitLog, segmentManager);
// Hard link file in cdc folder for realtime tracking
FileUtils.createHardLink(segment.logFile, segment.getCDCFile());
@@ -157,11 +170,8 @@ public CommitLogSegment createSegment()
*
* @param file segment file that is no longer in use.
*/
- @Override
- void handleReplayedSegment(final File file)
+ public void handleReplayedSegment(final File file)
{
- super.handleReplayedSegment(file);
-
// delete untracked cdc segment hard link files if their index files do not exist
File cdcFile = new File(DatabaseDescriptor.getCDCLogLocation(), file.getName());
File cdcIndexFile = new File(DatabaseDescriptor.getCDCLogLocation(), CommitLogDescriptor.fromFileName(file.getName()).cdcIndexFileName());
@@ -175,7 +185,7 @@ void handleReplayedSegment(final File file)
/**
* For use after replay when replayer hard-links / adds tracking of replayed segments
*/
- public void addCDCSize(long size)
+ void addCDCSize(long size)
{
cdcSizeTracker.addSize(size);
}
@@ -185,18 +195,18 @@ public void addCDCSize(long size)
* data in them and all segments archived into cdc_raw.
*
* Allows atomic increment/decrement of unflushed size, however only allows increment on flushed and requires a full
- * directory walk to determine any potential deletions by CDC consumer.
+ * directory walk to determine any potential deletions by an external CDC consumer.
*/
private static class CDCSizeTracker extends DirectorySizeCalculator
{
private final RateLimiter rateLimiter = RateLimiter.create(1000.0 / DatabaseDescriptor.getCDCDiskCheckInterval());
private ExecutorService cdcSizeCalculationExecutor;
- private CommitLogSegmentManagerCDC segmentManager;
+ private final CommitLogSegmentManager segmentManager;
- // Used instead of size during walk to remove chance of over-allocation
+ /** Used only in context of file tree walking thread; not read nor mutated outside this context */
private volatile long sizeInProgress = 0;
- CDCSizeTracker(CommitLogSegmentManagerCDC segmentManager, File path)
+ CDCSizeTracker(CommitLogSegmentManager segmentManager, File path)
{
super(path);
this.segmentManager = segmentManager;
@@ -215,9 +225,9 @@ public void start()
* Synchronous size recalculation on each segment creation/deletion call could lead to very long delays in new
* segment allocation, thus long delays in thread signaling to wake waiting allocation / writer threads.
*
- * This can be reached either from the segment management thread in ABstractCommitLogSegmentManager or from the
+ * This can be reached either from the segment management thread in CommitLogSegmentManager or from the
* size recalculation executor, so we synchronize on this object to reduce the race overlap window available for
- * size to get off.
+ * size to drift.
*
* Reference DirectorySizerBench for more information about performance of the directory size recalc.
*/
@@ -237,6 +247,10 @@ void processNewSegment(CommitLogSegment segment)
submitOverflowSizeRecalculation();
}
+ /**
+ * Upon segment discard, we need to adjust our known CDC consumption on disk based on whether or not this segment
+ * was flagged to be allowable for CDC.
+ */
void processDiscardedSegment(CommitLogSegment segment)
{
// See synchronization in CommitLogSegment.setCDCState
@@ -258,7 +272,13 @@ private long allowableCDCBytes()
return (long)DatabaseDescriptor.getCDCSpaceInMB() * 1024 * 1024;
}
- public void submitOverflowSizeRecalculation()
+ /**
+ * The overflow size calculation requires walking the flie tree and checking file size for all linked CDC
+ * files. As such, we do this async on the executor in the CDCSizeTracker instead of the context of the calling
+ * thread. While this can obviously introduce some delay / raciness in the calculation of CDC size consumed,
+ * the alternative of significantly long blocks for critical path CL allocation is unacceptable.
+ */
+ void submitOverflowSizeRecalculation()
{
try
{
@@ -274,9 +294,12 @@ private void recalculateOverflowSize()
{
rateLimiter.acquire();
calculateSize();
- CommitLogSegment allocatingFrom = segmentManager.allocatingFrom();
- if (allocatingFrom.getCDCState() == CDCState.FORBIDDEN)
- processNewSegment(allocatingFrom);
+ CommitLogSegment activeCommitLogSegment = segmentManager.getActiveSegment();
+ // In the event that the current segment is disallowed for CDC, re-check it as our size on disk may have
+ // reduced, thus allowing the segment to accept CDC writes. It's worth noting: this would spin on recalc
+ // endlessly if not for the rate limiter dropping looping calls on the floor.
+ if (activeCommitLogSegment.getCDCState() == CDCState.FORBIDDEN)
+ processNewSegment(activeCommitLogSegment);
}
private int defaultSegmentSize()
@@ -288,14 +311,14 @@ private void calculateSize()
{
try
{
- // The Arrays.stream approach is considerably slower on Windows than linux
+ // The Arrays.stream approach is considerably slower
sizeInProgress = 0;
Files.walkFileTree(path.toPath(), this);
size = sizeInProgress;
}
catch (IOException ie)
{
- CommitLog.instance.handleCommitError("Failed CDC Size Calculation", ie);
+ CommitLog.handleCommitError("Failed CDC Size Calculation", ie);
}
}
@@ -327,7 +350,7 @@ private long totalCDCSizeOnDisk()
* Only use for testing / validation that size tracker is working. Not for production use.
*/
@VisibleForTesting
- public long updateCDCTotalSize()
+ long updateCDCTotalSize()
{
cdcSizeTracker.submitOverflowSizeRecalculation();
@@ -336,7 +359,9 @@ public long updateCDCTotalSize()
{
Thread.sleep(DatabaseDescriptor.getCDCDiskCheckInterval() + 10);
}
- catch (InterruptedException e) {}
+ catch (InterruptedException e) {
+ // Expected in test context. no-op.
+ }
return cdcSizeTracker.totalCDCSizeOnDisk();
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorStandard.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorStandard.java
new file mode 100644
index 000000000000..f3d64bfbe922
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorStandard.java
@@ -0,0 +1,90 @@
+/*
+ * 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.commitlog;
+
+import java.io.File;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.io.util.FileUtils;
+
+/**
+ * This is a fairly simple form of a CommitLogSegmentAllocator.
+ */
+public class CommitLogSegmentAllocatorStandard implements CommitLogSegmentAllocator
+{
+ static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentAllocatorStandard.class);
+ private final CommitLogSegmentManager segmentManager;
+
+ public void start() {}
+ public void shutdown() {}
+
+ CommitLogSegmentAllocatorStandard(CommitLogSegmentManager segmentManager) {
+ this.segmentManager = segmentManager;
+ }
+
+ /**
+ * No extra processing required beyond deletion of the file once we have replayed it.
+ */
+ public void handleReplayedSegment(final File file) {
+ // (don't decrease managed size, since this was never a "live" segment)
+ logger.trace("(Unopened) segment {} is no longer needed and will be deleted now", file);
+ FileUtils.deleteWithConfirm(file);
+ }
+
+ public void discard(CommitLogSegment segment, boolean delete)
+ {
+ segment.close();
+ if (delete)
+ FileUtils.deleteWithConfirm(segment.logFile);
+ segmentManager.addSize(-segment.onDiskSize());
+ }
+
+ /**
+ * Reserve space in the current segment for the provided mutation or, if there isn't space available,
+ * create a new segment. allocate() is blocking until allocation succeeds as it waits on a signal in switchToNewSegment
+ *
+ * @param mutation mutation to allocate space for
+ * @param size total size of mutation (overhead + serialized size)
+ * @return the provided Allocation object
+ */
+ public CommitLogSegment.Allocation allocate(Mutation mutation, int size)
+ {
+ CommitLogSegment segment = segmentManager.getActiveSegment();
+
+ CommitLogSegment.Allocation alloc = segment.allocate(mutation, size);
+ // If we failed to allocate in the segment, prompt for a switch to a new segment and loop on re-attempt. This
+ // is expected to succeed or throw, since CommitLog allocation working is central to how a node operates.
+ while (alloc == null)
+ {
+ // Failed to allocate, so move to a new segment with enough room if possible.
+ segmentManager.switchToNewSegment(segment);
+ segment = segmentManager.getActiveSegment();
+ alloc = segment.allocate(mutation, size);
+ }
+
+ return alloc;
+ }
+
+ public CommitLogSegment createSegment()
+ {
+ return CommitLogSegment.createSegment(segmentManager.commitLog, segmentManager);
+ }
+}
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
similarity index 74%
rename from src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
rename to src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index fdbf7f661ff3..2c3878ac1917 100755
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -32,45 +32,41 @@
import net.nicoulaj.compilecommand.annotations.DontInline;
import org.apache.cassandra.concurrent.NamedThreadFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.db.*;
+import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.schema.TableId;
import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.WrappedRunnable;
import org.apache.cassandra.utils.concurrent.WaitQueue;
-import static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
-
/**
* Performs eager-creation of commit log segments in a background thread. All the
* public methods are thread safe.
*/
-public abstract class AbstractCommitLogSegmentManager
+public class CommitLogSegmentManager
{
- static final Logger logger = LoggerFactory.getLogger(AbstractCommitLogSegmentManager.class);
+ static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentManager.class);
/**
* Segment that is ready to be used. The management thread fills this and blocks until consumed.
*
* A single management thread produces this, and consumers are already synchronizing to make sure other work is
* performed atomically with consuming this. Volatile to make sure writes by the management thread become
- * visible (ordered/lazySet would suffice). Consumers (advanceAllocatingFrom and discardAvailableSegment) must
+ * visible (ordered/lazySet would suffice). Consumers (switchToNewSegment and discardAvailableSegment) must
* synchronize on 'this'.
*/
private volatile CommitLogSegment availableSegment = null;
private final WaitQueue segmentPrepared = new WaitQueue();
- /** Active segments, containing unflushed data. The tail of this queue is the one we allocate writes to */
- private final ConcurrentLinkedQueue activeSegments = new ConcurrentLinkedQueue<>();
-
- /**
- * The segment we are currently allocating commit log records to.
- *
- * Written by advanceAllocatingFrom which synchronizes on 'this'. Volatile to ensure reads get current value.
+ /** Segments that are still in memtables and not yet flushed to sstables.
+ * The tail of this queue is the one we allocate writes to.
*/
- private volatile CommitLogSegment allocatingFrom = null;
+ private final ConcurrentLinkedQueue unflushedSegments = new ConcurrentLinkedQueue<>();
+
+ /** The segment we are currently allocating commit log records to. */
+ private volatile CommitLogSegment activeSegment = null;
final String storageDirectory;
@@ -83,7 +79,7 @@ public abstract class AbstractCommitLogSegmentManager
private final AtomicLong size = new AtomicLong();
private Thread managerThread;
- protected final CommitLog commitLog;
+ final CommitLog commitLog;
private volatile boolean shutdown;
private final BooleanSupplier managerThreadWaitCondition = () -> (availableSegment == null && !atSegmentBufferLimit()) || shutdown;
private final WaitQueue managerThreadWaitQueue = new WaitQueue();
@@ -91,14 +87,20 @@ public abstract class AbstractCommitLogSegmentManager
private static final SimpleCachedBufferPool bufferPool =
new SimpleCachedBufferPool(DatabaseDescriptor.getCommitLogMaxCompressionBuffersInPool(), DatabaseDescriptor.getCommitLogSegmentSize());
- AbstractCommitLogSegmentManager(final CommitLog commitLog, String storageDirectory)
+ final CommitLogSegmentAllocator segmentAllocator;
+
+ CommitLogSegmentManager(final CommitLog commitLog, String storageDirectory)
{
this.commitLog = commitLog;
this.storageDirectory = storageDirectory;
+ this.segmentAllocator = DatabaseDescriptor.isCDCEnabled() ?
+ new CommitLogSegmentAllocatorCDC(this) :
+ new CommitLogSegmentAllocatorStandard(this);
}
void start()
{
+ segmentAllocator.start();
// The run loop for the manager thread
Runnable runnable = new WrappedRunnable()
{
@@ -110,7 +112,7 @@ public void runMayThrow() throws Exception
{
assert availableSegment == null;
logger.trace("No segments in reserve; creating a fresh one");
- availableSegment = createSegment();
+ availableSegment = segmentAllocator.createSegment();
if (shutdown)
{
// If shutdown() started and finished during segment creation, we are now left with a
@@ -127,8 +129,8 @@ public void runMayThrow() throws Exception
continue;
// Writing threads are not waiting for new segments, we can spend time on other tasks.
- // flush old Cfs if we're full
- maybeFlushToReclaim();
+ // Flush old Cfs if we're full.
+ flushIfOverLimit();
}
catch (Throwable t)
{
@@ -153,7 +155,7 @@ public void runMayThrow() throws Exception
managerThread.start();
// for simplicity, ensure the first segment is allocated before continuing
- advanceAllocatingFrom(null);
+ switchToNewSegment(null);
}
private boolean atSegmentBufferLimit()
@@ -161,75 +163,58 @@ private boolean atSegmentBufferLimit()
return CommitLogSegment.usesBufferPool(commitLog) && bufferPool.atLimit();
}
- private void maybeFlushToReclaim()
+ /**
+ * In the event we've overallocated (i.e. size on disk > limit in config), we want to trigger a flush of the number
+ * of required memtables to sstables in order to be able to reclaim some CL space on disk.
+ */
+ private void flushIfOverLimit()
{
- long unused = unusedCapacity();
- if (unused < 0)
+ if (overConfigDiskCapacity(0))
{
long flushingSize = 0;
List segmentsToRecycle = new ArrayList<>();
- for (CommitLogSegment segment : activeSegments)
+ for (CommitLogSegment segment : unflushedSegments)
{
- if (segment == allocatingFrom)
+ if (segment == activeSegment)
break;
flushingSize += segment.onDiskSize();
segmentsToRecycle.add(segment);
- if (flushingSize + unused >= 0)
+ if (!overConfigDiskCapacity((flushingSize)))
break;
}
- flushDataFrom(segmentsToRecycle, false);
+ flushTablesForSegments(segmentsToRecycle, false);
}
}
/**
- * Allocate a segment within this CLSM. Should either succeed or throw.
- */
- public abstract Allocation allocate(Mutation mutation, int size);
-
- /**
- * Hook to allow segment managers to track state surrounding creation of new segments. Onl perform as task submit
- * to segment manager so it's performed on segment management thread.
- */
- abstract CommitLogSegment createSegment();
-
- /**
- * Indicates that a segment file has been flushed and is no longer needed. Only perform as task submit to segment
- * manager so it's performend on segment management thread, or perform while segment management thread is shutdown
- * during testing resets.
- *
- * @param segment segment to be discarded
- * @param delete whether or not the segment is safe to be deleted.
- */
- abstract void discard(CommitLogSegment segment, boolean delete);
-
- /**
- * Advances the allocatingFrom pointer to the next prepared segment, but only if it is currently the segment provided.
+ * Advances the activeSegment pointer to the next prepared segment, but only if it is currently the segment provided.
*
* WARNING: Assumes segment management thread always succeeds in allocating a new segment or kills the JVM.
*/
@DontInline
- void advanceAllocatingFrom(CommitLogSegment old)
+ void switchToNewSegment(CommitLogSegment old)
{
while (true)
{
synchronized (this)
{
- // do this in a critical section so we can maintain the order of segment construction when moving to allocatingFrom/activeSegments
- if (allocatingFrom != old)
+ // do this in a critical section so we can maintain the order of segment construction when moving to activeSegment/unflushedSegments
+ if (activeSegment != old)
return;
// If a segment is ready, take it now, otherwise wait for the management thread to construct it.
if (availableSegment != null)
{
- // Success! Change allocatingFrom and activeSegments (which must be kept in order) before leaving
+ // Success! Change activeSegment and unflushedSegments (which must be kept in order) before leaving
// the critical section.
- activeSegments.add(allocatingFrom = availableSegment);
+ activeSegment = availableSegment;
+ unflushedSegments.add(activeSegment);
availableSegment = null;
break;
}
}
- awaitAvailableSegment(old);
+ awaitSegmentAllocation(old);
}
// Signal the management thread to prepare a new segment.
@@ -249,17 +234,22 @@ void advanceAllocatingFrom(CommitLogSegment old)
commitLog.requestExtraSync();
}
- void awaitAvailableSegment(CommitLogSegment currentAllocatingFrom)
+ /**
+ * Spins while waiting on next available segment's allocation, putting caller to sleep until the new segment is created.
+ * @param oldActiveSegment
+ */
+ private void awaitSegmentAllocation(CommitLogSegment oldActiveSegment)
{
do
{
WaitQueue.Signal prepared = segmentPrepared.register(commitLog.metrics.waitingOnSegmentAllocation.time());
- if (availableSegment == null && allocatingFrom == currentAllocatingFrom)
+ // No new segment created, and the active segment is the one we already know about. Time to sleep...
+ if (availableSegment == null && activeSegment == oldActiveSegment)
prepared.awaitUninterruptibly();
else
prepared.cancel();
}
- while (availableSegment == null && allocatingFrom == currentAllocatingFrom);
+ while (availableSegment == null && activeSegment == oldActiveSegment);
}
/**
@@ -269,9 +259,9 @@ void awaitAvailableSegment(CommitLogSegment currentAllocatingFrom)
*/
void forceRecycleAll(Iterable droppedTables)
{
- List segmentsToRecycle = new ArrayList<>(activeSegments);
+ List segmentsToRecycle = new ArrayList<>(unflushedSegments);
CommitLogSegment last = segmentsToRecycle.get(segmentsToRecycle.size() - 1);
- advanceAllocatingFrom(last);
+ switchToNewSegment(last);
// wait for the commit log modifications
last.waitForModifications();
@@ -281,26 +271,26 @@ void forceRecycleAll(Iterable droppedTables)
Keyspace.writeOrder.awaitNewBarrier();
// flush and wait for all CFs that are dirty in segments up-to and including 'last'
- Future> future = flushDataFrom(segmentsToRecycle, true);
+ Future> future = flushTablesForSegments(segmentsToRecycle, true);
try
{
future.get();
- for (CommitLogSegment segment : activeSegments)
+ for (CommitLogSegment segment : unflushedSegments)
for (TableId tableId : droppedTables)
segment.markClean(tableId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
// now recycle segments that are unused, as we may not have triggered a discardCompletedSegments()
// if the previous active segment was the only one to recycle (since an active segment isn't
// necessarily dirty, and we only call dCS after a flush).
- for (CommitLogSegment segment : activeSegments)
+ for (CommitLogSegment segment : unflushedSegments)
{
if (segment.isUnused())
archiveAndDiscard(segment);
}
- CommitLogSegment first;
- if ((first = activeSegments.peek()) != null && first.id <= last.id)
+ CommitLogSegment first = unflushedSegments.peek();
+ if (first != null && first.id <= last.id)
logger.error("Failed to force-recycle all segments; at least one segment is still in use with dirty CFs.");
}
catch (Throwable t)
@@ -318,11 +308,11 @@ void forceRecycleAll(Iterable droppedTables)
void archiveAndDiscard(final CommitLogSegment segment)
{
boolean archiveSuccess = commitLog.archiver.maybeWaitForArchiving(segment.getName());
- if (!activeSegments.remove(segment))
+ if (!unflushedSegments.remove(segment))
return; // already discarded
// if archiving (command) was not successful then leave the file alone. don't delete or recycle.
logger.debug("Segment {} is no longer active and will be deleted {}", segment, archiveSuccess ? "now" : "by the archive script");
- discard(segment, archiveSuccess);
+ segmentAllocator.discard(segment, archiveSuccess);
}
/**
@@ -332,9 +322,7 @@ void archiveAndDiscard(final CommitLogSegment segment)
*/
void handleReplayedSegment(final File file)
{
- // (don't decrease managed size, since this was never a "live" segment)
- logger.trace("(Unopened) segment {} is no longer needed and will be deleted now", file);
- FileUtils.deleteWithConfirm(file);
+ segmentAllocator.handleReplayedSegment(file);
}
/**
@@ -354,12 +342,16 @@ public long onDiskSize()
return size.get();
}
- private long unusedCapacity()
+ /**
+ * We offset by the amount we've planned to flush with to allow for selective calculation up front of how much to flush
+ */
+ private boolean overConfigDiskCapacity(long toBeFlushed)
{
long total = DatabaseDescriptor.getTotalCommitlogSpaceInMB() * 1024 * 1024;
- long currentSize = size.get();
+ long currentSize = size.get() + toBeFlushed;
logger.trace("Total active commitlog segment space used is {} out of {}", currentSize, total);
- return total - currentSize;
+ // TODO: Consider whether to do >=. Original logic strictly equated with > from CASSANDRA-9095
+ return currentSize > total;
}
/**
@@ -367,7 +359,7 @@ private long unusedCapacity()
*
* @return a Future that will finish when all the flushes are complete.
*/
- private Future> flushDataFrom(List segments, boolean force)
+ private Future> flushTablesForSegments(List segments, boolean force)
{
if (segments.isEmpty())
return Futures.immediateFuture(null);
@@ -393,7 +385,7 @@ else if (!flushes.containsKey(dirtyTableId))
final ColumnFamilyStore cfs = Keyspace.open(metadata.keyspace).getColumnFamilyStore(dirtyTableId);
// can safely call forceFlush here as we will only ever block (briefly) for other attempts to flush,
// no deadlock possibility since switchLock removal
- flushes.put(dirtyTableId, force ? cfs.forceFlush() : cfs.forceFlush(maxCommitLogPosition));
+ flushes.put(dirtyTableId, force ? cfs.forceFlushToSSTable() : cfs.forceFlushToSSTable(maxCommitLogPosition));
}
}
}
@@ -405,7 +397,7 @@ else if (!flushes.containsKey(dirtyTableId))
* Stops CL, for testing purposes. DO NOT USE THIS OUTSIDE OF TESTS.
* Only call this after the AbstractCommitLogService is shut down.
*/
- public void stopUnsafe(boolean deleteSegments)
+ void stopUnsafe(boolean deleteSegments)
{
logger.debug("CLSM closing and clearing existing commit log segments...");
@@ -419,9 +411,9 @@ public void stopUnsafe(boolean deleteSegments)
throw new RuntimeException(e);
}
- for (CommitLogSegment segment : activeSegments)
+ for (CommitLogSegment segment : unflushedSegments)
closeAndDeleteSegmentUnsafe(segment, deleteSegments);
- activeSegments.clear();
+ unflushedSegments.clear();
size.set(0L);
@@ -435,7 +427,7 @@ void awaitManagementTasksCompletion()
{
if (availableSegment == null && !atSegmentBufferLimit())
{
- awaitAvailableSegment(allocatingFrom);
+ awaitSegmentAllocation(activeSegment);
}
}
@@ -446,7 +438,7 @@ private void closeAndDeleteSegmentUnsafe(CommitLogSegment segment, boolean delet
{
try
{
- discard(segment, delete);
+ segmentAllocator.discard(segment, delete);
}
catch (AssertionError ignored)
{
@@ -455,13 +447,15 @@ private void closeAndDeleteSegmentUnsafe(CommitLogSegment segment, boolean delet
}
/**
- * Initiates the shutdown process for the management thread.
+ * Initiates the shutdown process for the management thread and segment allocator.
*/
public void shutdown()
{
assert !shutdown;
shutdown = true;
+ segmentAllocator.shutdown();
+
// Release the management thread and delete prepared segment.
// Do not block as another thread may claim the segment (this can happen during unit test initialization).
discardAvailableSegment();
@@ -488,19 +482,21 @@ public void awaitTermination() throws InterruptedException
managerThread.join();
managerThread = null;
- for (CommitLogSegment segment : activeSegments)
+ for (CommitLogSegment segment : unflushedSegments)
segment.close();
bufferPool.shutdown();
}
/**
- * @return a read-only collection of the active commit log segments
+ * @return a read-only collection of all active and unflushed segments in the system. In this context, "Flushed" is
+ * referring to "memtable / CF flushed to sstables", not whether or not the CommitLogSegment itself is flushed via
+ * fsync.
*/
@VisibleForTesting
- public Collection getActiveSegments()
+ public Collection getSegmentsForUnflushedTables()
{
- return Collections.unmodifiableCollection(activeSegments);
+ return Collections.unmodifiableCollection(unflushedSegments);
}
/**
@@ -508,7 +504,7 @@ public Collection getActiveSegments()
*/
CommitLogPosition getCurrentPosition()
{
- return allocatingFrom.getCurrentCommitLogPosition();
+ return activeSegment.getCurrentCommitLogPosition();
}
/**
@@ -518,8 +514,8 @@ CommitLogPosition getCurrentPosition()
*/
public void sync(boolean flush) throws IOException
{
- CommitLogSegment current = allocatingFrom;
- for (CommitLogSegment segment : getActiveSegments())
+ CommitLogSegment current = activeSegment;
+ for (CommitLogSegment segment : getSegmentsForUnflushedTables())
{
// Do not sync segments that became active after sync started.
if (segment.id > current.id)
@@ -550,10 +546,17 @@ void notifyBufferFreed()
wakeManager();
}
- /** Read-only access to current segment for subclasses. */
- CommitLogSegment allocatingFrom()
+ /**
+ * Pass-through call to allocator. Allocates a mutation in the active CommitLogSegment.
+ */
+ CommitLogSegment.Allocation allocate(Mutation mutation, int size) {
+ return segmentAllocator.allocate(mutation, size);
+ }
+
+ /** Read-only access to current segment for package usage. */
+ CommitLogSegment getActiveSegment()
{
- return allocatingFrom;
+ return activeSegment;
}
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
deleted file mode 100644
index b9bd744da1a5..000000000000
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.commitlog;
-
-import java.io.File;
-
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.io.util.FileUtils;
-
-public class CommitLogSegmentManagerStandard extends AbstractCommitLogSegmentManager
-{
- public CommitLogSegmentManagerStandard(final CommitLog commitLog, String storageDirectory)
- {
- super(commitLog, storageDirectory);
- }
-
- public void discard(CommitLogSegment segment, boolean delete)
- {
- segment.close();
- if (delete)
- FileUtils.deleteWithConfirm(segment.logFile);
- addSize(-segment.onDiskSize());
- }
-
- /**
- * Reserve space in the current segment for the provided mutation or, if there isn't space available,
- * create a new segment. allocate() is blocking until allocation succeeds as it waits on a signal in advanceAllocatingFrom
- *
- * @param mutation mutation to allocate space for
- * @param size total size of mutation (overhead + serialized size)
- * @return the provided Allocation object
- */
- public CommitLogSegment.Allocation allocate(Mutation mutation, int size)
- {
- CommitLogSegment segment = allocatingFrom();
-
- CommitLogSegment.Allocation alloc;
- while ( null == (alloc = segment.allocate(mutation, size)) )
- {
- // failed to allocate, so move to a new segment with enough room
- advanceAllocatingFrom(segment);
- segment = allocatingFrom();
- }
-
- return alloc;
- }
-
- public CommitLogSegment createSegment()
- {
- return CommitLogSegment.createSegment(commitLog, this);
- }
-}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java
index e23a915ba355..fd8ff3078ddb 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java
@@ -21,59 +21,64 @@
import java.nio.ByteBuffer;
import java.util.Iterator;
import java.util.zip.CRC32;
+import javax.annotation.concurrent.NotThreadSafe;
import javax.crypto.Cipher;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.AbstractIterator;
+import org.apache.cassandra.db.commitlog.CommitLogReadHandler.CommitLogReadErrorReason;
+import org.apache.cassandra.db.commitlog.CommitLogReadHandler.CommitLogReadException;
import org.apache.cassandra.db.commitlog.EncryptedFileSegmentInputStream.ChunkProvider;
-import org.apache.cassandra.db.commitlog.CommitLogReadHandler.*;
import org.apache.cassandra.io.FSReadError;
import org.apache.cassandra.io.compress.ICompressor;
import org.apache.cassandra.io.util.FileDataInput;
import org.apache.cassandra.io.util.FileSegmentInputStream;
import org.apache.cassandra.io.util.RandomAccessReader;
import org.apache.cassandra.schema.CompressionParams;
-import org.apache.cassandra.security.EncryptionUtils;
import org.apache.cassandra.security.EncryptionContext;
+import org.apache.cassandra.security.EncryptionUtils;
import org.apache.cassandra.utils.ByteBufferUtil;
import static org.apache.cassandra.db.commitlog.CommitLogSegment.SYNC_MARKER_SIZE;
import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;
/**
- * Read each sync section of a commit log, iteratively.
+ * Read each sync section of a commit log, iteratively. Can be run in either one-shot or resumable mode. In resumable,
+ * we snapshot the start position of any successful SyncSegment deserialization with the expectation that some reads will
+ * land in partially written segments and need to be rolled back to the start of that segment and repeated on further
+ * mutation serialization (specifically in encrypted or compressed contexts).
*/
+@NotThreadSafe
public class CommitLogSegmentReader implements Iterable
{
- private final CommitLogReadHandler handler;
- private final CommitLogDescriptor descriptor;
- private final RandomAccessReader reader;
+ private final ResumableCommitLogReader parent;
private final Segmenter segmenter;
- private final boolean tolerateTruncation;
- /**
- * ending position of the current sync section.
- */
+ /** A special SyncSegment we use to indicate / keep our iterators open on a read we intend to resume */
+ static final SyncSegment RESUMABLE_SENTINEL = new SyncSegment(null, -1, -1, -1, false);
+
+ /** Ending position of the current sync section. */
protected int end;
- protected CommitLogSegmentReader(CommitLogReadHandler handler,
- CommitLogDescriptor descriptor,
- RandomAccessReader reader,
- boolean tolerateTruncation)
+ /**
+ * Rather than relying on a formal Builder, this constructs the appropriate type of segment reader (memmap, encrypted,
+ * compressed) based on the type stored in the descriptor.
+ *
+ * Note: If ever using this object directly in a test, ensure you set the {@link ResumableCommitLogReader#offsetLimit}
+ * before attempting to use this reader or iteration will never advance.
+ */
+ CommitLogSegmentReader(ResumableCommitLogReader parent)
{
- this.handler = handler;
- this.descriptor = descriptor;
- this.reader = reader;
- this.tolerateTruncation = tolerateTruncation;
-
- end = (int) reader.getFilePointer();
- if (descriptor.getEncryptionContext().isEnabled())
- segmenter = new EncryptedSegmenter(descriptor, reader);
- else if (descriptor.compression != null)
- segmenter = new CompressedSegmenter(descriptor, reader);
+ this.parent = parent;
+
+ end = (int) parent.rawReader.getFilePointer();
+ if (parent.descriptor.getEncryptionContext().isEnabled())
+ segmenter = new EncryptedSegmenter(parent.descriptor, parent);
+ else if (parent.descriptor.compression != null)
+ segmenter = new CompressedSegmenter(parent.descriptor, parent);
else
- segmenter = new NoOpSegmenter(reader);
+ segmenter = new NoOpSegmenter(parent.rawReader);
}
public Iterator iterator()
@@ -81,36 +86,75 @@ public Iterator iterator()
return new SegmentIterator();
}
+ /** Will return endOfData() or our resumable sentinel depending on what mode the iterator is being used in */
protected class SegmentIterator extends AbstractIterator
{
protected SyncSegment computeNext()
{
+ // A couple sanity checks that we're in a good state
+ if (parent.offsetLimit == Integer.MIN_VALUE)
+ throw new RuntimeException("Attempted to use a CommitLogSegmentReader with an uninitialized ResumableCommitLogReader parent.");
+
+ // Since this could be mis-used by client app parsing code, keep it RTE instead of assertion.
+ if (parent.isClosed)
+ throw new RuntimeException("Attempted to use a closed ResumableCommitLogReader.");
+
while (true)
{
try
{
final int currentStart = end;
- end = readSyncMarker(descriptor, currentStart, reader);
- if (end == -1)
+
+ // Segmenters need to know our original state to appropriately roll back on snapshot restore
+ segmenter.stageSnapshot();
+ end = readSyncMarker(parent.descriptor, currentStart, parent.rawReader);
+
+ if (parent.isPartial())
{
- return endOfData();
+ // Revert our SegmentIterator's state to beginning of last completed SyncSegment read on a partial read.
+ if (end == -1 || end > parent.offsetLimit)
+ {
+ segmenter.revertToSnapshot();
+ end = (int)parent.rawReader.getFilePointer();
+ return RESUMABLE_SENTINEL;
+ }
+ // Flag our RR's data as exhausted if we've hit the end of our reader but think this is partial.
+ else if (end >= parent.rawReader.length())
+ {
+ parent.readToExhaustion = true;
+ }
}
- if (end > reader.length())
+ // Iterate on a non-resumable read.
+ else
{
- // the CRC was good (meaning it was good when it was written and still looks legit), but the file is truncated now.
- // try to grab and use as much of the file as possible, which might be nothing if the end of the file truly is corrupt
- end = (int) reader.length();
+ if (end == -1)
+ {
+ // We only transition to endOfData if we're doing a non-resumable (i.e. read to end) read,
+ // since it leaves this iterator in a non-reusable state.
+ return endOfData();
+ }
+ else if (end > parent.rawReader.length())
+ {
+ // the CRC was good (meaning it was good when it was written and still looks legit), but the file is truncated now.
+ // try to grab and use as much of the file as possible, which might be nothing if the end of the file truly is corrupt
+ end = (int) parent.rawReader.length();
+ }
}
+
+ // Retain the starting point of this SyncSegment in case we need to roll back a future read to this point.
+ segmenter.takeSnapshot();
+
+ // Passed the gauntlet. The next segment is cleanly ready for read.
return segmenter.nextSegment(currentStart + SYNC_MARKER_SIZE, end);
}
catch(CommitLogSegmentReader.SegmentReadException e)
{
try
{
- handler.handleUnrecoverableError(new CommitLogReadException(
- e.getMessage(),
- CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR,
- !e.invalidCrc && tolerateTruncation));
+ parent.readHandler.handleUnrecoverableError(new CommitLogReadException(
+ e.getMessage(),
+ CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR,
+ !e.invalidCrc && parent.tolerateTruncation));
}
catch (IOException ioe)
{
@@ -121,12 +165,12 @@ protected SyncSegment computeNext()
{
try
{
- boolean tolerateErrorsInSection = tolerateTruncation & segmenter.tolerateSegmentErrors(end, reader.length());
+ boolean tolerateErrorsInSection = parent.tolerateTruncation & segmenter.tolerateSegmentErrors(end, parent.rawReader.length());
// if no exception is thrown, the while loop will continue
- handler.handleUnrecoverableError(new CommitLogReadException(
- e.getMessage(),
- CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR,
- tolerateErrorsInSection));
+ parent.readHandler.handleUnrecoverableError(new CommitLogReadException(
+ e.getMessage(),
+ CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR,
+ tolerateErrorsInSection));
}
catch (IOException ioe)
{
@@ -137,13 +181,13 @@ protected SyncSegment computeNext()
}
}
+ /**
+ * @return length of this sync segment, -1 if at or beyond the end of file.
+ */
private int readSyncMarker(CommitLogDescriptor descriptor, int offset, RandomAccessReader reader) throws IOException
{
if (offset > reader.length() - SYNC_MARKER_SIZE)
- {
- // There was no room in the segment to write a final header. No data could be present here.
return -1;
- }
reader.seek(offset);
CRC32 crc = new CRC32();
updateChecksumInt(crc, (int) (descriptor.id & 0xFFFFFFFFL));
@@ -180,23 +224,24 @@ public SegmentReadException(String msg, boolean invalidCrc)
}
}
+ /** The logical unit of data we sync across and read across in CommitLogs. */
public static class SyncSegment
{
/** the 'buffer' to replay commit log data from */
public final FileDataInput input;
/** offset in file where this section begins. */
- public final int fileStartPosition;
+ final int fileStartPosition;
/** offset in file where this section ends. */
- public final int fileEndPosition;
+ final int fileEndPosition;
/** the logical ending position of the buffer */
- public final int endPosition;
+ final int endPosition;
- public final boolean toleratesErrorsInSection;
+ final boolean toleratesErrorsInSection;
- public SyncSegment(FileDataInput input, int fileStartPosition, int fileEndPosition, int endPosition, boolean toleratesErrorsInSection)
+ SyncSegment(FileDataInput input, int fileStartPosition, int fileEndPosition, int endPosition, boolean toleratesErrorsInSection)
{
this.input = input;
this.fileStartPosition = fileStartPosition;
@@ -208,6 +253,8 @@ public SyncSegment(FileDataInput input, int fileStartPosition, int fileEndPositi
/**
* Derives the next section of the commit log to be replayed. Section boundaries are derived from the commit log sync markers.
+ * Allows snapshot and resume from snapshot functionality to revert to a "last known good segment" in the event of
+ * a partial read on an a file being actively written.
*/
interface Segmenter
{
@@ -228,13 +275,29 @@ default boolean tolerateSegmentErrors(int segmentEndPosition, long fileLength)
{
return segmentEndPosition >= fileLength || segmentEndPosition < 0;
}
+
+ /** Holds snapshot data in temporary variables to be finalized when we determine a SyncSegment is fully written */
+ void stageSnapshot();
+
+ /** Finalizes snapshot staged in stageSnapshot */
+ void takeSnapshot();
+
+ /** Reverts the segmenter to the previously held position. Allows for resumable reads to rollback when they occur
+ * in the middle of a SyncSegment. This can be called repeatedly if we have multiple attempts to partially read
+ * on an incomplete SyncSegment. */
+ void revertToSnapshot();
+
+ /** Visible for debugging only */
+ long getSnapshot();
}
static class NoOpSegmenter implements Segmenter
{
private final RandomAccessReader reader;
+ private long snapshotPosition = Long.MIN_VALUE;
+ private long stagedSnapshot = Long.MIN_VALUE;
- public NoOpSegmenter(RandomAccessReader reader)
+ NoOpSegmenter(RandomAccessReader reader)
{
this.reader = reader;
}
@@ -249,54 +312,114 @@ public boolean tolerateSegmentErrors(int end, long length)
{
return true;
}
+
+ public void stageSnapshot()
+ {
+ stagedSnapshot = reader.getFilePointer();
+ // Deal with edge case of initial read attempt being before SyncSegment completion
+ if (snapshotPosition == Long.MIN_VALUE)
+ takeSnapshot();
+ }
+
+ public void takeSnapshot()
+ {
+ snapshotPosition = stagedSnapshot;
+ }
+
+ public void revertToSnapshot()
+ {
+ reader.seek(snapshotPosition);
+ }
+
+ public long getSnapshot()
+ {
+ return snapshotPosition;
+ }
}
static class CompressedSegmenter implements Segmenter
{
private final ICompressor compressor;
- private final RandomAccessReader reader;
+ /** We store a reference to a ResumableReader in the event it needs to re-init and swap out the underlying reader */
+ private final ResumableCommitLogReader parent;
private byte[] compressedBuffer;
private byte[] uncompressedBuffer;
private long nextLogicalStart;
- public CompressedSegmenter(CommitLogDescriptor desc, RandomAccessReader reader)
+ private long stagedLogicalStart = Long.MIN_VALUE;
+ private long stagedReaderLocation = Long.MIN_VALUE;
+ private long snapshotLogicalStart = Long.MIN_VALUE;
+ private long snapshotReaderLocation = Long.MIN_VALUE;
+
+ CompressedSegmenter(CommitLogDescriptor desc, ResumableCommitLogReader parent)
{
- this(CompressionParams.createCompressor(desc.compression), reader);
+ this(CompressionParams.createCompressor(desc.compression), parent);
}
- public CompressedSegmenter(ICompressor compressor, RandomAccessReader reader)
+ CompressedSegmenter(ICompressor compressor, ResumableCommitLogReader parent)
{
this.compressor = compressor;
- this.reader = reader;
+ this.parent = parent;
compressedBuffer = new byte[0];
uncompressedBuffer = new byte[0];
- nextLogicalStart = reader.getFilePointer();
+ nextLogicalStart = parent.rawReader.getFilePointer();
}
@SuppressWarnings("resource")
public SyncSegment nextSegment(final int startPosition, final int nextSectionStartPosition) throws IOException
{
- reader.seek(startPosition);
- int uncompressedLength = reader.readInt();
+ parent.rawReader.seek(startPosition);
+ int uncompressedLength = parent.rawReader.readInt();
- int compressedLength = nextSectionStartPosition - (int)reader.getPosition();
+ int compressedLength = nextSectionStartPosition - (int)parent.rawReader.getPosition();
if (compressedLength > compressedBuffer.length)
compressedBuffer = new byte[(int) (1.2 * compressedLength)];
- reader.readFully(compressedBuffer, 0, compressedLength);
+ parent.rawReader.readFully(compressedBuffer, 0, compressedLength);
if (uncompressedLength > uncompressedBuffer.length)
uncompressedBuffer = new byte[(int) (1.2 * uncompressedLength)];
int count = compressor.uncompress(compressedBuffer, 0, compressedLength, uncompressedBuffer, 0);
nextLogicalStart += SYNC_MARKER_SIZE;
- FileDataInput input = new FileSegmentInputStream(ByteBuffer.wrap(uncompressedBuffer, 0, count), reader.getPath(), nextLogicalStart);
+ FileDataInput input = new FileSegmentInputStream(ByteBuffer.wrap(uncompressedBuffer, 0, count), parent.rawReader.getPath(), nextLogicalStart);
nextLogicalStart += uncompressedLength;
- return new SyncSegment(input, startPosition, nextSectionStartPosition, (int)nextLogicalStart, tolerateSegmentErrors(nextSectionStartPosition, reader.length()));
+ return new SyncSegment(input, startPosition, nextSectionStartPosition, (int)nextLogicalStart, tolerateSegmentErrors(nextSectionStartPosition, parent.rawReader.length()));
+ }
+
+ public void stageSnapshot()
+ {
+ stagedLogicalStart = nextLogicalStart;
+ stagedReaderLocation = parent.rawReader.getFilePointer();
+
+ // In our default 0 case on a segment w/out anything yet to read, we want to stage the first valid location
+ // we've seen, else a resume will kick us to a bad value
+ if (snapshotLogicalStart == Long.MIN_VALUE)
+ takeSnapshot();
+ }
+
+ /** Since {@link #nextLogicalStart} is mutated during decompression but relied upon for decompression, we need
+ * to both snapshot and revert that along with the reader's position. */
+ public void takeSnapshot()
+ {
+ snapshotLogicalStart = stagedLogicalStart;
+ snapshotReaderLocation = stagedReaderLocation;
+ }
+
+ public void revertToSnapshot()
+ {
+ nextLogicalStart = snapshotLogicalStart;
+ parent.rawReader.seek(snapshotReaderLocation);
+ }
+
+ public long getSnapshot()
+ {
+ return snapshotReaderLocation;
}
}
static class EncryptedSegmenter implements Segmenter
{
- private final RandomAccessReader reader;
+ /** We store a reference to a ResumableReader in the event it needs to re-init and swap out the underlying reader */
+ private final ResumableCommitLogReader parent;
private final ICompressor compressor;
private final Cipher cipher;
@@ -315,18 +438,21 @@ static class EncryptedSegmenter implements Segmenter
private long currentSegmentEndPosition;
private long nextLogicalStart;
- public EncryptedSegmenter(CommitLogDescriptor descriptor, RandomAccessReader reader)
+ private long stagedSnapshotPosition;
+ private long snapshotPosition;
+
+ EncryptedSegmenter(CommitLogDescriptor descriptor, ResumableCommitLogReader parent)
{
- this(reader, descriptor.getEncryptionContext());
+ this(parent, descriptor.getEncryptionContext());
}
@VisibleForTesting
- EncryptedSegmenter(final RandomAccessReader reader, EncryptionContext encryptionContext)
+ EncryptedSegmenter(final ResumableCommitLogReader parent, EncryptionContext encryptionContext)
{
- this.reader = reader;
+ this.parent = parent;
decryptedBuffer = ByteBuffer.allocate(0);
compressor = encryptionContext.getCompressor();
- nextLogicalStart = reader.getFilePointer();
+ nextLogicalStart = parent.rawReader.getFilePointer();
try
{
@@ -334,21 +460,21 @@ public EncryptedSegmenter(CommitLogDescriptor descriptor, RandomAccessReader rea
}
catch (IOException ioe)
{
- throw new FSReadError(ioe, reader.getPath());
+ throw new FSReadError(ioe, parent.rawReader.getPath());
}
chunkProvider = () -> {
- if (reader.getFilePointer() >= currentSegmentEndPosition)
+ if (parent.rawReader.getFilePointer() >= currentSegmentEndPosition)
return ByteBufferUtil.EMPTY_BYTE_BUFFER;
try
{
- decryptedBuffer = EncryptionUtils.decrypt(reader, decryptedBuffer, true, cipher);
+ decryptedBuffer = EncryptionUtils.decrypt(parent.rawReader, decryptedBuffer, true, cipher);
uncompressedBuffer = EncryptionUtils.uncompress(decryptedBuffer, uncompressedBuffer, true, compressor);
return uncompressedBuffer;
}
catch (IOException e)
{
- throw new FSReadError(e, reader.getPath());
+ throw new FSReadError(e, parent.rawReader.getPath());
}
};
}
@@ -356,13 +482,35 @@ public EncryptedSegmenter(CommitLogDescriptor descriptor, RandomAccessReader rea
@SuppressWarnings("resource")
public SyncSegment nextSegment(int startPosition, int nextSectionStartPosition) throws IOException
{
- int totalPlainTextLength = reader.readInt();
+ int totalPlainTextLength = parent.rawReader.readInt();
currentSegmentEndPosition = nextSectionStartPosition - 1;
nextLogicalStart += SYNC_MARKER_SIZE;
- FileDataInput input = new EncryptedFileSegmentInputStream(reader.getPath(), nextLogicalStart, 0, totalPlainTextLength, chunkProvider);
+ FileDataInput input = new EncryptedFileSegmentInputStream(parent.rawReader.getPath(), nextLogicalStart, 0, totalPlainTextLength, chunkProvider);
nextLogicalStart += totalPlainTextLength;
- return new SyncSegment(input, startPosition, nextSectionStartPosition, (int)nextLogicalStart, tolerateSegmentErrors(nextSectionStartPosition, reader.length()));
+ return new SyncSegment(input, startPosition, nextSectionStartPosition, (int)nextLogicalStart, tolerateSegmentErrors(nextSectionStartPosition, parent.rawReader.length()));
+ }
+
+ public void stageSnapshot()
+ {
+ stagedSnapshotPosition = parent.rawReader.getFilePointer();
+ if (snapshotPosition == Long.MIN_VALUE)
+ takeSnapshot();
+ }
+
+ public void takeSnapshot()
+ {
+ snapshotPosition = stagedSnapshotPosition;
+ }
+
+ public void revertToSnapshot()
+ {
+ parent.rawReader.seek(snapshotPosition);
+ }
+
+ public long getSnapshot()
+ {
+ return snapshotPosition;
}
}
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java b/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java
index d5e61137d842..c74e8430b218 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java
@@ -41,7 +41,7 @@ public class CompressedSegment extends FileDirectSegment
/**
* Constructs a new segment file.
*/
- CompressedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
+ CompressedSegment(CommitLog commitLog, CommitLogSegmentManager manager)
{
super(commitLog, manager);
this.compressor = commitLog.configuration.getCompressor();
@@ -59,7 +59,7 @@ void write(int startMarker, int nextMarker)
int contentStart = startMarker + SYNC_MARKER_SIZE;
int length = nextMarker - contentStart;
// The length may be 0 when the segment is being closed.
- assert length > 0 || length == 0 && !isStillAllocating();
+ assert length > 0 || length == 0 && !hasRoom();
try
{
diff --git a/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java b/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java
index 21b7c11fb052..b699da438290 100644
--- a/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java
@@ -64,7 +64,7 @@ public class EncryptedSegment extends FileDirectSegment
private final EncryptionContext encryptionContext;
private final Cipher cipher;
- public EncryptedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
+ public EncryptedSegment(CommitLog commitLog, CommitLogSegmentManager manager)
{
super(commitLog, manager);
this.encryptionContext = commitLog.configuration.getEncryptionContext();
@@ -101,7 +101,7 @@ void write(int startMarker, int nextMarker)
int contentStart = startMarker + SYNC_MARKER_SIZE;
final int length = nextMarker - contentStart;
// The length may be 0 when the segment is being closed.
- assert length > 0 || length == 0 && !isStillAllocating();
+ assert length > 0 || length == 0 && !hasRoom();
final ICompressor compressor = encryptionContext.getCompressor();
final int blockSize = encryptionContext.getChunkLength();
diff --git a/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java b/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java
index d5431f875b5f..d82fd10d40a6 100644
--- a/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java
@@ -31,7 +31,7 @@ public abstract class FileDirectSegment extends CommitLogSegment
{
volatile long lastWrittenPos = 0;
- FileDirectSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
+ FileDirectSegment(CommitLog commitLog, CommitLogSegmentManager manager)
{
super(commitLog, manager);
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
index 6ecdbd3c7764..9009e4cd2250 100644
--- a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
@@ -22,6 +22,9 @@
import java.nio.MappedByteBuffer;
import java.nio.channels.FileChannel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.util.FileUtils;
@@ -35,12 +38,14 @@
*/
public class MemoryMappedSegment extends CommitLogSegment
{
+ private static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentReader.class);
+
/**
* Constructs a new segment file.
*
* @param commitLog the commit log it will be used with.
*/
- MemoryMappedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
+ MemoryMappedSegment(CommitLog commitLog, CommitLogSegmentManager manager)
{
super(commitLog, manager);
// mark the initial sync marker as uninitialised
diff --git a/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java
new file mode 100644
index 000000000000..ac925bb5f36c
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java
@@ -0,0 +1,272 @@
+/*
+ * 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.commitlog;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.text.MessageFormat;
+import java.util.Iterator;
+import java.util.Optional;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * A state holder for a potentially resumable read. As we want to resume our reading with existing file pointers, buffers,
+ * and sentinels without re-opening a file and re-decompressing or decrypting, we store references to a {@link CommitLogSegmentReader},
+ * to a {@link RandomAccessReader}, and to an iterator of {@link CommitLogSegmentReader.SyncSegment} here for re-use.
+ *
+ * This serves dual purpose as an API endpoint and logical state holder to pop out our handles across multiple reads
+ * while minimizing pollution to core CommitLogReader code implementation.
+ *
+ * _Mandatory_ usage of this API is as follows:
+ * 0-N) {@link #readPartial(int limit)}
+ * 1) {@link #readToCompletion()}
+ * NOTE: neither of these callse will {@link #close} this reader. try-with-resources is the correct usage.
+ * to correctly close out.
+ *
+ * As this is intended to be used both in an internal C* state as well as by external users looking to read CommitLogSegments,
+ * we allow construction to fail gracefully and indicate usability through {@link #isClosed()}.
+ */
+@NotThreadSafe
+public class ResumableCommitLogReader implements AutoCloseable
+{
+ private static final Logger logger = LoggerFactory.getLogger(CommitLogReader.class);
+
+ /** We hold a reference to these so we can re-use them for subsequent descriptor parsing on resumed reads */
+ final File segmentFile;
+ final CommitLogDescriptor descriptor;
+ private final CommitLogReader commitLogReader;
+ final CommitLogReadHandler readHandler;
+ final boolean tolerateTruncation;
+
+ /** Can be re-initialized if re-reading a reader w/compression enabled and we're at our known limit. */
+ RandomAccessReader rawReader;
+
+ /** We allow the users to determine whether or not the system should continue on various forms of read failure. As
+ * such, we allow resumable readers to be constructed even if they are unusable to the end-user. */
+ boolean isClosed = false;
+
+ /** Separate sentinel to indicate whether we have read to completion on our underlying file. Flagged by SegmentReader
+ * We use this during {@link #reBufferData()} to determine whether or not to recreate our underlying RAR in the compressed
+ * case.
+ */
+ boolean readToExhaustion = false;
+
+ /** Minimum position before which we completely skip CommitLogSegments */
+ final CommitLogPosition minPosition;
+
+ /** Sentinel used to limit reads */
+ final int mutationLimit;
+
+ /** We cache, snapshot, and revert position inside our {@link CommitLogSegmentReader.SegmentIterator#computeNext} calls
+ * to keep the user-facing API simple */
+ @Nullable
+ Iterator activeIterator;
+
+ @Nullable
+ private CommitLogSegmentReader segmentReader;
+
+ /** Raw file offset at which we will stop iterating and processing mutations on a read */
+ int offsetLimit = Integer.MIN_VALUE;
+
+ public ResumableCommitLogReader(File commitLogSegment, CommitLogReadHandler readHandler) throws IOException, NullPointerException
+ {
+ this(commitLogSegment, readHandler, CommitLogPosition.NONE, CommitLogReader.ALL_MUTATIONS, true);
+ }
+
+ public ResumableCommitLogReader(File commitLogSegment,
+ CommitLogReadHandler readHandler,
+ CommitLogPosition minPosition,
+ int mutationLimit,
+ boolean tolerateTruncation) throws IOException, NullPointerException
+ {
+ this.segmentFile = commitLogSegment;
+ this.commitLogReader = new CommitLogReader();
+ this.readHandler = readHandler;
+ this.mutationLimit = mutationLimit;
+ this.minPosition = minPosition;
+ this.tolerateTruncation = tolerateTruncation;
+
+ Pair, Integer> header = CommitLogReader.readCommitLogDescriptor(readHandler,
+ commitLogSegment,
+ tolerateTruncation);
+ // Totally fail out if we fail to parse this CommitLogSegment descriptor
+ if (!header.left.isPresent())
+ throw new RuntimeException(MessageFormat.format("Failed to parse the CommitLogDescriptor from {0}", commitLogSegment));
+ descriptor = header.left.get();
+
+ if (shouldSkipSegmentId(new File(descriptor.fileName()), descriptor, minPosition))
+ {
+ close();
+ }
+ else
+ {
+ try
+ {
+ this.rawReader = RandomAccessReader.open(commitLogSegment);
+ rawReader.seek(header.right);
+
+ // This is where we grab and old open our handles if we succeed
+ segmentReader = CommitLogReader.getCommitLogSegmentReader(this);
+ if (segmentReader != null)
+ this.activeIterator = segmentReader.iterator();
+ }
+ finally
+ {
+ if (segmentReader == null)
+ close();
+ }
+ }
+ }
+
+ /**
+ * Performs a partial CommitLogSegment read. Closes down this resumable reader on read error.
+ *
+ * @param readLimit How far to read into the file before stopping.
+ */
+ public void readPartial(int readLimit) throws IOException
+ {
+ if (isClosed)
+ {
+ logger.warn("Attempted to use invalid ResumableCommitLogReader for file {}. Ignoring.", descriptor.fileName());
+ return;
+ }
+
+ if (readLimit <= offsetLimit)
+ {
+ logger.warn("Attempted to resume reading a commit log but used already read offset: {}", readLimit);
+ return;
+ }
+ offsetLimit = readLimit;
+ rebufferAndRead();
+ }
+
+ /** Reads to end of file from current cached offset. */
+ public void readToCompletion() throws IOException
+ {
+ if (isClosed)
+ {
+ logger.warn("Attempted to use invalid ResumableCommitLogReader for file {}. Ignoring.", descriptor.fileName());
+ return;
+ }
+ offsetLimit = CommitLogReader.READ_TO_END_OF_FILE;
+ rebufferAndRead();
+ }
+
+ public void close()
+ {
+ isClosed = true;
+ if (rawReader != null)
+ rawReader.close();
+ segmentReader = null;
+ activeIterator = null;
+ }
+
+ public boolean isClosed()
+ {
+ return isClosed;
+ }
+
+ /**
+ * When we have compression enabled, RandomAccessReader's have CompressionMetadata to indicate the end of their file
+ * length. For our purposes, this means we have some difficulty in re-using previously constructed underlying buffers
+ * for decompression and reading, so if the underlying file length has changed because a file is actively being written
+ * and we've exhausted the current data we know about, we close out our RAR and construct a new one with the new
+ * {@link org.apache.cassandra.io.compress.CompressionMetadata}. While it would arguably be better to extend the
+ * hierarchy to have a rebuffering compressed segment, YAGNI for now. The added gc pressure from this + overhead
+ * on closing and re-opening RAR's should be restricted to non-node partial/resumed CL reading cases which we expect
+ * to have very different properties than critical path log replay on a running node, for example.
+ */
+ private void reBufferData() throws FileNotFoundException
+ {
+ if (readToExhaustion)
+ {
+ long toSeek = rawReader.getPosition();
+ this.rawReader.close();
+ if (!segmentFile.exists())
+ throw new FileNotFoundException(String.format("Attempting to reBufferData but underlying file cannot be found: {}",
+ segmentFile.getAbsolutePath()));
+ this.rawReader = RandomAccessReader.open(segmentFile);
+ this.rawReader.seek(toSeek);
+ }
+ else
+ {
+ rawReader.reBuffer();
+ }
+ }
+
+ /** Performs the read operation and closes down this reader on exception. */
+ private void rebufferAndRead() throws RuntimeException, IOException
+ {
+ reBufferData();
+
+ try
+ {
+ commitLogReader.internalReadCommitLogSegment(this);
+ }
+ catch (RuntimeException | IOException e)
+ {
+ close();
+ throw e;
+ }
+ }
+
+ /** Any segment with id >= minPosition.segmentId is a candidate for read. */
+ private boolean shouldSkipSegmentId(File file, CommitLogDescriptor desc, CommitLogPosition minPosition)
+ {
+ logger.debug("Reading {} (CL version {}, messaging version {}, compression {})",
+ file.getPath(),
+ desc.version,
+ desc.getMessagingVersion(),
+ desc.compression);
+
+ if (minPosition.segmentId > desc.id)
+ {
+ logger.trace("Skipping read of fully-flushed {}", file);
+ return true;
+ }
+ return false;
+ }
+
+ /** Flag to indicate how the {@link CommitLogSegmentReader.SegmentIterator} should behave on failure to compute next
+ * segments.
+ */
+ boolean isPartial()
+ {
+ return offsetLimit != CommitLogReader.READ_TO_END_OF_FILE;
+ }
+
+ @Override
+ public String toString()
+ {
+ return new StringBuilder()
+ .append("File: ").append(descriptor.fileName()).append(", ")
+ .append("minPos: ").append(minPosition).append(", ")
+ .append("offsetLimit: ").append(offsetLimit).append(", ")
+ .append("readerPos: ").append(rawReader.getPosition()).append(", ")
+ .append("activeIter: ").append(activeIterator)
+ .toString();
+ }
+}
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 74c154d51905..ad494b15da06 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -21,8 +21,6 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.SerializationHeader;
@@ -412,8 +410,8 @@ else if (CompactionController.getFullyExpiredSSTables(cfs, Collections.singleton
ranges.add(new Range<>(overlap.first.getToken(), overlap.last.getToken()));
long remainingKeys = keys - sstable.estimatedKeysForRanges(ranges);
// next, calculate what percentage of columns we have within those keys
- long columns = sstable.getEstimatedColumnCount().mean() * remainingKeys;
- double remainingColumnsRatio = ((double) columns) / (sstable.getEstimatedColumnCount().count() * sstable.getEstimatedColumnCount().mean());
+ long columns = sstable.getEstimatedCellPerPartitionCount().mean() * remainingKeys;
+ double remainingColumnsRatio = ((double) columns) / (sstable.getEstimatedCellPerPartitionCount().count() * sstable.getEstimatedCellPerPartitionCount().mean());
// return if we still expect to have droppable tombstones in rest of columns
return remainingColumnsRatio * droppableRatio > tombstoneThreshold;
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index 59bba0a5cc8b..e1b0f3258359 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -19,26 +19,21 @@
import java.util.*;
import java.util.function.LongPredicate;
-import java.util.function.Predicate;
-
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.db.Memtable;
-import org.apache.cassandra.db.rows.UnfilteredRowIterator;
import com.google.common.base.Predicates;
import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.RateLimiter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.db.*;
import org.apache.cassandra.db.partitions.Partition;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.util.FileDataInput;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.schema.CompactionParams.TombstoneOption;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.*;
import org.apache.cassandra.utils.AlwaysPresentFilter;
import org.apache.cassandra.utils.OverlapIterator;
import org.apache.cassandra.utils.concurrent.Refs;
@@ -48,13 +43,12 @@
/**
* Manage compaction options.
*/
-public class CompactionController implements AutoCloseable
+public class CompactionController extends AbstractCompactionController
{
private static final Logger logger = LoggerFactory.getLogger(CompactionController.class);
private static final String NEVER_PURGE_TOMBSTONES_PROPERTY = Config.PROPERTY_PREFIX + "never_purge_tombstones";
static final boolean NEVER_PURGE_TOMBSTONES = Boolean.getBoolean(NEVER_PURGE_TOMBSTONES_PROPERTY);
- public final ColumnFamilyStore cfs;
private final boolean compactingRepaired;
// note that overlapIterator and overlappingSSTables will be null if NEVER_PURGE_TOMBSTONES is set - this is a
// good thing so that noone starts using them and thinks that if overlappingSSTables is empty, there
@@ -64,11 +58,8 @@ public class CompactionController implements AutoCloseable
private final Iterable compacting;
private final RateLimiter limiter;
private final long minTimestamp;
- final TombstoneOption tombstoneOption;
final Map openDataFiles = new HashMap<>();
- public final int gcBefore;
-
protected CompactionController(ColumnFamilyStore cfs, int maxValue)
{
this(cfs, null, maxValue);
@@ -82,13 +73,10 @@ public CompactionController(ColumnFamilyStore cfs, Set compacting
public CompactionController(ColumnFamilyStore cfs, Set compacting, int gcBefore, RateLimiter limiter, TombstoneOption tombstoneOption)
{
- assert cfs != null;
- this.cfs = cfs;
- this.gcBefore = gcBefore;
+ super(cfs, gcBefore, tombstoneOption);
this.compacting = compacting;
this.limiter = limiter;
compactingRepaired = compacting != null && compacting.stream().allMatch(SSTableReader::isRepaired);
- this.tombstoneOption = tombstoneOption;
this.minTimestamp = compacting != null && !compacting.isEmpty() // check needed for test
? compacting.stream().mapToLong(SSTableReader::getMinTimestamp).min().getAsLong()
: 0;
@@ -246,16 +234,6 @@ public static Set getFullyExpiredSSTables(ColumnFamilyStore cfSto
return getFullyExpiredSSTables(cfStore, compacting, overlapping, gcBefore, false);
}
- public String getKeyspace()
- {
- return cfs.keyspace.getName();
- }
-
- public String getColumnFamily()
- {
- return cfs.name;
- }
-
/**
* @param key
* @return a predicate for whether tombstones marked for deletion at the given time for the given partition are
@@ -263,6 +241,7 @@ public String getColumnFamily()
* containing his partition and not participating in the compaction. This means there isn't any data in those
* sstables that might still need to be suppressed by a tombstone at this timestamp.
*/
+ @Override
public LongPredicate getPurgeEvaluator(DecoratedKey key)
{
if (NEVER_PURGE_TOMBSTONES || !compactingRepaired() || cfs.getNeverPurgeTombstones())
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
index 1c56a87bc82d..789d1eeeb5c8 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
@@ -57,7 +57,7 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
private static final long UNFILTERED_TO_UPDATE_PROGRESS = 100;
private final OperationType type;
- private final CompactionController controller;
+ private final AbstractCompactionController controller;
private final List scanners;
private final ImmutableSet sstables;
private final int nowInSec;
@@ -77,13 +77,13 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
private final UnfilteredPartitionIterator compacted;
private final ActiveCompactionsTracker activeCompactions;
- public CompactionIterator(OperationType type, List scanners, CompactionController controller, int nowInSec, UUID compactionId)
+ public CompactionIterator(OperationType type, List scanners, AbstractCompactionController controller, int nowInSec, UUID compactionId)
{
this(type, scanners, controller, nowInSec, compactionId, ActiveCompactionsTracker.NOOP);
}
@SuppressWarnings("resource") // We make sure to close mergedIterator in close() and CompactionIterator is itself an AutoCloseable
- public CompactionIterator(OperationType type, List scanners, CompactionController controller, int nowInSec, UUID compactionId, ActiveCompactionsTracker activeCompactions)
+ public CompactionIterator(OperationType type, List scanners, AbstractCompactionController controller, int nowInSec, UUID compactionId, ActiveCompactionsTracker activeCompactions)
{
this.controller = controller;
this.type = type;
@@ -259,14 +259,14 @@ public String toString()
private class Purger extends PurgeFunction
{
- private final CompactionController controller;
+ private final AbstractCompactionController controller;
private DecoratedKey currentKey;
private LongPredicate purgeEvaluator;
private long compactedUnfiltered;
- private Purger(CompactionController controller, int nowInSec)
+ private Purger(AbstractCompactionController controller, int nowInSec)
{
super(nowInSec, controller.gcBefore, controller.compactingRepaired() ? Integer.MAX_VALUE : Integer.MIN_VALUE,
controller.cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones(),
@@ -510,10 +510,10 @@ private DeletionTime updateOpenDeletionTime(DeletionTime openDeletionTime, Unfil
*/
private static class GarbageSkipper extends Transformation
{
- final CompactionController controller;
+ final AbstractCompactionController controller;
final boolean cellLevelGC;
- private GarbageSkipper(CompactionController controller)
+ private GarbageSkipper(AbstractCompactionController controller)
{
this.controller = controller;
cellLevelGC = controller.tombstoneOption == TombstoneOption.CELL;
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index d38770180c37..bb1d585e00b9 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -346,6 +346,7 @@ public BackgroundCompactionCandidate getBackgroundCompactionCandidate(ColumnFami
@SuppressWarnings("resource")
private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs, OperationType operationType) throws ExecutionException, InterruptedException
{
+ logger.info("Starting {} for {}.{}", operationType, cfs.keyspace.getName(), cfs.getTableName());
List transactions = new ArrayList<>();
List> futures = new ArrayList<>();
try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType))
@@ -387,6 +388,7 @@ public Object call() throws Exception
}
FBUtilities.waitOnFutures(futures);
assert compacting.originals().isEmpty();
+ logger.info("Finished {} for {}.{} successfully", operationType, cfs.keyspace.getName(), cfs.getTableName());
return AllSSTableOpStatus.SUCCESSFUL;
}
finally
@@ -402,7 +404,7 @@ public Object call() throws Exception
}
Throwable fail = Throwables.close(null, transactions);
if (fail != null)
- logger.error("Failed to cleanup lifecycle transactions", fail);
+ logger.error("Failed to cleanup lifecycle transactions ({} for {}.{})", operationType, cfs.keyspace.getName(), cfs.getTableName(), fail);
}
}
@@ -527,7 +529,34 @@ public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jo
public Iterable filterSSTables(LifecycleTransaction transaction)
{
List sortedSSTables = Lists.newArrayList(transaction.originals());
- Collections.sort(sortedSSTables, SSTableReader.sizeComparator);
+ Iterator sstableIter = sortedSSTables.iterator();
+ int totalSSTables = 0;
+ int skippedSStables = 0;
+ while (sstableIter.hasNext())
+ {
+ SSTableReader sstable = sstableIter.next();
+ boolean needsCleanupFull = needsCleanup(sstable, fullRanges);
+ boolean needsCleanupTransient = needsCleanup(sstable, transientRanges);
+ //If there are no ranges for which the table needs cleanup either due to lack of intersection or lack
+ //of the table being repaired.
+ totalSSTables++;
+ if (!needsCleanupFull && (!needsCleanupTransient || !sstable.isRepaired()))
+ {
+ logger.debug("Skipping {} ([{}, {}]) for cleanup; all rows should be kept. Needs cleanup full ranges: {} Needs cleanup transient ranges: {} Repaired: {}",
+ sstable,
+ sstable.first.getToken(),
+ sstable.last.getToken(),
+ needsCleanupFull,
+ needsCleanupTransient,
+ sstable.isRepaired());
+ sstableIter.remove();
+ transaction.cancel(sstable);
+ skippedSStables++;
+ }
+ }
+ logger.info("Skipping cleanup for {}/{} sstables for {}.{} since they are fully contained in owned ranges (full ranges: {}, transient ranges: {})",
+ skippedSStables, totalSSTables, cfStore.keyspace.getName(), cfStore.getTableName(), fullRanges, transientRanges);
+ sortedSSTables.sort(SSTableReader.sizeComparator);
return sortedSSTables;
}
@@ -817,14 +846,15 @@ public void performMaximal(final ColumnFamilyStore cfStore, boolean splitOutput)
FBUtilities.waitOnFutures(submitMaximal(cfStore, getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds()), splitOutput));
}
+ @SuppressWarnings("resource") // the tasks are executed in parallel on the executor, making sure that they get closed
public List> submitMaximal(final ColumnFamilyStore cfStore, final int gcBefore, boolean splitOutput)
{
// here we compute the task off the compaction executor, so having that present doesn't
// confuse runWithCompactionsDisabled -- i.e., we don't want to deadlock ourselves, waiting
// for ourselves to finish/acknowledge cancellation before continuing.
- final Collection tasks = cfStore.getCompactionStrategyManager().getMaximalTasks(gcBefore, splitOutput);
+ CompactionTasks tasks = cfStore.getCompactionStrategyManager().getMaximalTasks(gcBefore, splitOutput);
- if (tasks == null)
+ if (tasks.isEmpty())
return Collections.emptyList();
List> futures = new ArrayList<>();
@@ -850,42 +880,42 @@ protected void runMayThrow()
if (nonEmptyTasks > 1)
logger.info("Major compaction will not result in a single sstable - repaired and unrepaired data is kept separate and compaction runs per data_file_directory.");
-
return futures;
}
public void forceCompactionForTokenRange(ColumnFamilyStore cfStore, Collection> ranges)
{
- final Collection tasks = cfStore.runWithCompactionsDisabled(() ->
- {
- Collection sstables = sstablesInBounds(cfStore, ranges);
- if (sstables == null || sstables.isEmpty())
- {
- logger.debug("No sstables found for the provided token range");
- return null;
- }
- return cfStore.getCompactionStrategyManager().getUserDefinedTasks(sstables, getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds()));
- }, (sstable) -> new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges), false, false, false);
-
- if (tasks == null)
- return;
-
- Runnable runnable = new WrappedRunnable()
- {
- protected void runMayThrow()
+ Callable taskCreator = () -> {
+ Collection sstables = sstablesInBounds(cfStore, ranges);
+ if (sstables == null || sstables.isEmpty())
{
- for (AbstractCompactionTask task : tasks)
- if (task != null)
- task.execute(active);
+ logger.debug("No sstables found for the provided token range");
+ return CompactionTasks.empty();
}
+ return cfStore.getCompactionStrategyManager().getUserDefinedTasks(sstables, getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds()));
};
- if (executor.isShutdown())
+ try (CompactionTasks tasks = cfStore.runWithCompactionsDisabled(taskCreator,
+ (sstable) -> new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges),
+ false,
+ false,
+ false))
{
- logger.info("Compaction executor has shut down, not submitting task");
- return;
+ if (tasks.isEmpty())
+ return;
+
+ Runnable runnable = new WrappedRunnable()
+ {
+ protected void runMayThrow()
+ {
+ for (AbstractCompactionTask task : tasks)
+ if (task != null)
+ task.execute(active);
+ }
+ };
+
+ FBUtilities.waitOnFuture(executor.submitIfRunning(runnable, "force compaction for token range"));
}
- FBUtilities.waitOnFuture(executor.submit(runnable));
}
private static Collection sstablesInBounds(ColumnFamilyStore cfs, Collection> tokenRangeCollection)
@@ -990,7 +1020,7 @@ public Future> submitUserDefined(final ColumnFamilyStore cfs, final Collection
{
Runnable runnable = new WrappedRunnable()
{
- protected void runMayThrow()
+ protected void runMayThrow() throws Exception
{
// look up the sstables now that we're on the compaction executor, so we don't try to re-compact
// something that was already being compacted earlier.
@@ -1015,11 +1045,13 @@ protected void runMayThrow()
}
else
{
- List tasks = cfs.getCompactionStrategyManager().getUserDefinedTasks(sstables, gcBefore);
- for (AbstractCompactionTask task : tasks)
+ try (CompactionTasks tasks = cfs.getCompactionStrategyManager().getUserDefinedTasks(sstables, gcBefore))
{
- if (task != null)
- task.execute(active);
+ for (AbstractCompactionTask task : tasks)
+ {
+ if (task != null)
+ task.execute(active);
+ }
}
}
}
@@ -1171,16 +1203,7 @@ private void doCleanupOne(final ColumnFamilyStore cfs,
{
txn.obsoleteOriginals();
txn.finish();
- return;
- }
-
- boolean needsCleanupFull = needsCleanup(sstable, fullRanges);
- boolean needsCleanupTransient = needsCleanup(sstable, transientRanges);
- //If there are no ranges for which the table needs cleanup either due to lack of intersection or lack
- //of the table being repaired.
- if (!needsCleanupFull && (!needsCleanupTransient || !sstable.isRepaired()))
- {
- logger.trace("Skipping {} for cleanup; all rows should be kept. Needs cleanup full ranges: {} Needs cleanup transient ranges: {} Repaired: {}", sstable, needsCleanupFull, needsCleanupTransient, sstable.isRepaired());
+ logger.info("SSTable {} ([{}, {}]) does not intersect the owned ranges ({}), dropping it", sstable, sstable.first.getToken(), sstable.last.getToken(), allRanges);
return;
}
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index b97864185965..fd4dbeb7d8ca 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -29,7 +29,6 @@
import java.util.List;
import java.util.Set;
import java.util.UUID;
-import java.util.concurrent.Callable;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@@ -956,34 +955,27 @@ private void validateForCompaction(Iterable input)
}
}
- public Collection getMaximalTasks(final int gcBefore, final boolean splitOutput)
+ public CompactionTasks getMaximalTasks(final int gcBefore, final boolean splitOutput)
{
maybeReloadDiskBoundaries();
// runWithCompactionsDisabled cancels active compactions and disables them, then we are able
// to make the repaired/unrepaired strategies mark their own sstables as compacting. Once the
// sstables are marked the compactions are re-enabled
- return cfs.runWithCompactionsDisabled(new Callable>()
- {
- @Override
- public Collection call()
+ return cfs.runWithCompactionsDisabled(() -> {
+ List tasks = new ArrayList<>();
+ readLock.lock();
+ try
{
- List tasks = new ArrayList<>();
- readLock.lock();
- try
- {
- for (AbstractStrategyHolder holder : holders)
- {
- tasks.addAll(holder.getMaximalTasks(gcBefore, splitOutput));
- }
- }
- finally
+ for (AbstractStrategyHolder holder : holders)
{
- readLock.unlock();
+ tasks.addAll(holder.getMaximalTasks(gcBefore, splitOutput));
}
- if (tasks.isEmpty())
- return null;
- return tasks;
}
+ finally
+ {
+ readLock.unlock();
+ }
+ return CompactionTasks.create(tasks);
}, false, false);
}
@@ -996,7 +988,7 @@ public Collection call()
* @param gcBefore gc grace period, throw away tombstones older than this
* @return a list of compaction tasks corresponding to the sstables requested
*/
- public List getUserDefinedTasks(Collection sstables, int gcBefore)
+ public CompactionTasks getUserDefinedTasks(Collection sstables, int gcBefore)
{
maybeReloadDiskBoundaries();
List ret = new ArrayList<>();
@@ -1008,7 +1000,7 @@ public List getUserDefinedTasks(Collection implements AutoCloseable
+{
+ @SuppressWarnings("resource")
+ private static final CompactionTasks EMPTY = new CompactionTasks(Collections.emptyList());
+
+ private final Collection tasks;
+
+ private CompactionTasks(Collection tasks)
+ {
+ this.tasks = tasks;
+ }
+
+ public static CompactionTasks create(Collection tasks)
+ {
+ if (tasks == null || tasks.isEmpty())
+ return EMPTY;
+ return new CompactionTasks(tasks);
+ }
+
+ public static CompactionTasks empty()
+ {
+ return EMPTY;
+ }
+
+ public Iterator iterator()
+ {
+ return tasks.iterator();
+ }
+
+ public int size()
+ {
+ return tasks.size();
+ }
+
+ public void close()
+ {
+ try
+ {
+ FBUtilities.closeAll(tasks.stream().map(task -> task.transaction).collect(Collectors.toList()));
+ }
+ catch (Exception e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/db/context/CounterContext.java b/src/java/org/apache/cassandra/db/context/CounterContext.java
index 29dc3f087cd4..01c2f1d65dd0 100644
--- a/src/java/org/apache/cassandra/db/context/CounterContext.java
+++ b/src/java/org/apache/cassandra/db/context/CounterContext.java
@@ -629,7 +629,7 @@ public ByteBuffer markLocalToBeCleared(ByteBuffer context)
ByteBuffer marked = ByteBuffer.allocate(context.remaining());
marked.putShort(marked.position(), (short) (count * -1));
- ByteBufferUtil.arrayCopy(context,
+ ByteBufferUtil.copyBytes(context,
context.position() + HEADER_SIZE_LENGTH,
marked,
marked.position() + HEADER_SIZE_LENGTH,
@@ -668,7 +668,7 @@ public ByteBuffer clearAllLocal(ByteBuffer context)
cleared.putShort(cleared.position() + HEADER_SIZE_LENGTH + i * HEADER_ELT_LENGTH, globalShardIndexes.get(i));
int origHeaderLength = headerLength(context);
- ByteBufferUtil.arrayCopy(context,
+ ByteBufferUtil.copyBytes(context,
context.position() + origHeaderLength,
cleared,
cleared.position() + headerLength(cleared),
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
index ddc7b1c2cc21..d0cc5143a5df 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
@@ -26,6 +26,7 @@
import org.apache.cassandra.db.marshal.CollectionType;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.exceptions.UnknownColumnException;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.schema.ColumnMetadata;
@@ -196,7 +197,7 @@ public ColumnSubselection deserialize(DataInputPlus in, int version, TableMetada
// deserialization. The column will be ignore later on anyway.
column = metadata.getDroppedColumn(name);
if (column == null)
- throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
+ throw new UnknownColumnException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
}
Kind kind = Kind.values()[in.readUnsignedByte()];
diff --git a/src/java/org/apache/cassandra/db/filter/DataLimits.java b/src/java/org/apache/cassandra/db/filter/DataLimits.java
index 50baf2170add..3a766e0c2aa8 100644
--- a/src/java/org/apache/cassandra/db/filter/DataLimits.java
+++ b/src/java/org/apache/cassandra/db/filter/DataLimits.java
@@ -69,6 +69,12 @@ public UnfilteredRowIterator filter(UnfilteredRowIterator iter,
{
return iter;
}
+
+ @Override
+ public PartitionIterator filter(PartitionIterator iter, int nowInSec, boolean countPartitionsWithOnlyStaticData, boolean enforceStrictLiveness)
+ {
+ return iter;
+ }
};
// We currently deal with distinct queries by querying full partitions but limiting the result at 1 row per
@@ -440,7 +446,7 @@ public float estimateTotalResults(ColumnFamilyStore cfs)
{
// TODO: we should start storing stats on the number of rows (instead of the number of cells, which
// is what getMeanColumns returns)
- float rowsPerPartition = ((float) cfs.getMeanColumns()) / cfs.metadata().regularColumns().size();
+ float rowsPerPartition = ((float) cfs.getMeanEstimatedCellPerPartitionCount()) / cfs.metadata().regularColumns().size();
return rowsPerPartition * (cfs.estimateKeys());
}
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index ac4c69f7a6d2..e3423ff58008 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -360,7 +360,7 @@ public static ByteBuffer build(boolean isStatic, ByteBuffer... buffers)
{
ByteBufferUtil.writeShortLength(out, bb.remaining());
int toCopy = bb.remaining();
- ByteBufferUtil.arrayCopy(bb, bb.position(), out, out.position(), toCopy);
+ ByteBufferUtil.copyBytes(bb, bb.position(), out, out.position(), toCopy);
out.position(out.position() + toCopy);
out.put((byte) 0);
}
diff --git a/src/java/org/apache/cassandra/db/marshal/DecimalType.java b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
index b98bf009cdfc..110dc0e924e6 100644
--- a/src/java/org/apache/cassandra/db/marshal/DecimalType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
@@ -20,6 +20,7 @@
import java.math.BigDecimal;
import java.math.BigInteger;
import java.math.MathContext;
+import java.math.RoundingMode;
import java.nio.ByteBuffer;
import org.apache.cassandra.cql3.CQL3Type;
@@ -34,6 +35,10 @@
public class DecimalType extends NumberType
{
public static final DecimalType instance = new DecimalType();
+ private static final int MIN_SCALE = 32;
+ private static final int MIN_SIGNIFICANT_DIGITS = MIN_SCALE;
+ private static final int MAX_SCALE = 1000;
+ private static final MathContext MAX_PRECISION = new MathContext(10000);
DecimalType() {super(ComparisonType.CUSTOM);} // singleton
@@ -142,27 +147,41 @@ protected BigDecimal toBigDecimal(ByteBuffer value)
public ByteBuffer add(NumberType> leftType, ByteBuffer left, NumberType> rightType, ByteBuffer right)
{
- return decompose(leftType.toBigDecimal(left).add(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+ return decompose(leftType.toBigDecimal(left).add(rightType.toBigDecimal(right), MAX_PRECISION));
}
public ByteBuffer substract(NumberType> leftType, ByteBuffer left, NumberType> rightType, ByteBuffer right)
{
- return decompose(leftType.toBigDecimal(left).subtract(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+ return decompose(leftType.toBigDecimal(left).subtract(rightType.toBigDecimal(right), MAX_PRECISION));
}
public ByteBuffer multiply(NumberType> leftType, ByteBuffer left, NumberType> rightType, ByteBuffer right)
{
- return decompose(leftType.toBigDecimal(left).multiply(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+ return decompose(leftType.toBigDecimal(left).multiply(rightType.toBigDecimal(right), MAX_PRECISION));
}
public ByteBuffer divide(NumberType> leftType, ByteBuffer left, NumberType> rightType, ByteBuffer right)
{
- return decompose(leftType.toBigDecimal(left).divide(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+ BigDecimal leftOperand = leftType.toBigDecimal(left);
+ BigDecimal rightOperand = rightType.toBigDecimal(right);
+
+ // Predict position of first significant digit in the quotient.
+ // Note: it is possible to improve prediction accuracy by comparing first significant digits in operands
+ // but it requires additional computations so this step is omitted
+ int quotientFirstDigitPos = (leftOperand.precision() - leftOperand.scale()) - (rightOperand.precision() - rightOperand.scale());
+
+ int scale = MIN_SIGNIFICANT_DIGITS - quotientFirstDigitPos;
+ scale = Math.max(scale, leftOperand.scale());
+ scale = Math.max(scale, rightOperand.scale());
+ scale = Math.max(scale, MIN_SCALE);
+ scale = Math.min(scale, MAX_SCALE);
+
+ return decompose(leftOperand.divide(rightOperand, scale, RoundingMode.HALF_UP).stripTrailingZeros());
}
public ByteBuffer mod(NumberType> leftType, ByteBuffer left, NumberType> rightType, ByteBuffer right)
{
- return decompose(leftType.toBigDecimal(left).remainder(rightType.toBigDecimal(right), MathContext.DECIMAL128));
+ return decompose(leftType.toBigDecimal(left).remainder(rightType.toBigDecimal(right)));
}
public ByteBuffer negate(ByteBuffer input)
diff --git a/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java b/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java
deleted file mode 100644
index cc4b41041ce3..000000000000
--- a/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.monitoring;
-
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.Config;
-
-/**
- * This is an approximation of System.currentTimeInMillis(). It updates its
- * time value at periodic intervals of CHECK_INTERVAL_MS milliseconds
- * (currently 10 milliseconds by default). It can be used as a faster alternative
- * to System.currentTimeInMillis() every time an imprecision of a few milliseconds
- * can be accepted.
- */
-public class ApproximateTime
-{
- private static final Logger logger = LoggerFactory.getLogger(ApproximateTime.class);
- private static final int CHECK_INTERVAL_MS = Math.max(5, Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "approximate_time_precision_ms", "10")));
-
- private static volatile long time = System.currentTimeMillis();
- static
- {
- logger.info("Scheduling approximate time-check task with a precision of {} milliseconds", CHECK_INTERVAL_MS);
- ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(() -> time = System.currentTimeMillis(),
- CHECK_INTERVAL_MS,
- CHECK_INTERVAL_MS,
- TimeUnit.MILLISECONDS);
- }
-
- public static long currentTimeMillis()
- {
- return time;
- }
-
- public static long precision()
- {
- return 2 * CHECK_INTERVAL_MS;
- }
-
-}
diff --git a/src/java/org/apache/cassandra/db/monitoring/Monitorable.java b/src/java/org/apache/cassandra/db/monitoring/Monitorable.java
index c9bf94e08f46..10bd10438aa5 100644
--- a/src/java/org/apache/cassandra/db/monitoring/Monitorable.java
+++ b/src/java/org/apache/cassandra/db/monitoring/Monitorable.java
@@ -21,9 +21,9 @@
public interface Monitorable
{
String name();
- long constructionTime();
- long timeout();
- long slowTimeout();
+ long creationTimeNanos();
+ long timeoutNanos();
+ long slowTimeoutNanos();
boolean isInProgress();
boolean isAborted();
diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java b/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java
index 48c815270498..a6e7947b23f1 100644
--- a/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java
+++ b/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java
@@ -18,13 +18,15 @@
package org.apache.cassandra.db.monitoring;
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+
public abstract class MonitorableImpl implements Monitorable
{
private MonitoringState state;
private boolean isSlow;
- private long constructionTime = -1;
- private long timeout;
- private long slowTimeout;
+ private long approxCreationTimeNanos = -1;
+ private long timeoutNanos;
+ private long slowTimeoutNanos;
private boolean isCrossNode;
protected MonitorableImpl()
@@ -38,23 +40,23 @@ protected MonitorableImpl()
* is too complex, it would require passing new parameters to all serializers
* or specializing the serializers to accept these message properties.
*/
- public void setMonitoringTime(long constructionTime, boolean isCrossNode, long timeout, long slowTimeout)
+ public void setMonitoringTime(long approxCreationTimeNanos, boolean isCrossNode, long timeoutNanos, long slowTimeoutNanos)
{
- assert constructionTime >= 0;
- this.constructionTime = constructionTime;
+ assert approxCreationTimeNanos >= 0;
+ this.approxCreationTimeNanos = approxCreationTimeNanos;
this.isCrossNode = isCrossNode;
- this.timeout = timeout;
- this.slowTimeout = slowTimeout;
+ this.timeoutNanos = timeoutNanos;
+ this.slowTimeoutNanos = slowTimeoutNanos;
}
- public long constructionTime()
+ public long creationTimeNanos()
{
- return constructionTime;
+ return approxCreationTimeNanos;
}
- public long timeout()
+ public long timeoutNanos()
{
- return timeout;
+ return timeoutNanos;
}
public boolean isCrossNode()
@@ -62,9 +64,9 @@ public boolean isCrossNode()
return isCrossNode;
}
- public long slowTimeout()
+ public long slowTimeoutNanos()
{
- return slowTimeout;
+ return slowTimeoutNanos;
}
public boolean isInProgress()
@@ -95,8 +97,8 @@ public boolean abort()
{
if (state == MonitoringState.IN_PROGRESS)
{
- if (constructionTime >= 0)
- MonitoringTask.addFailedOperation(this, ApproximateTime.currentTimeMillis());
+ if (approxCreationTimeNanos >= 0)
+ MonitoringTask.addFailedOperation(this, approxTime.now());
state = MonitoringState.ABORTED;
return true;
@@ -109,8 +111,8 @@ public boolean complete()
{
if (state == MonitoringState.IN_PROGRESS)
{
- if (isSlow && slowTimeout > 0 && constructionTime >= 0)
- MonitoringTask.addSlowOperation(this, ApproximateTime.currentTimeMillis());
+ if (isSlow && slowTimeoutNanos > 0 && approxCreationTimeNanos >= 0)
+ MonitoringTask.addSlowOperation(this, approxTime.now());
state = MonitoringState.COMPLETED;
return true;
@@ -121,15 +123,15 @@ public boolean complete()
private void check()
{
- if (constructionTime < 0 || state != MonitoringState.IN_PROGRESS)
+ if (approxCreationTimeNanos < 0 || state != MonitoringState.IN_PROGRESS)
return;
- long elapsed = ApproximateTime.currentTimeMillis() - constructionTime;
+ long minElapsedNanos = (approxTime.now() - approxCreationTimeNanos) - approxTime.error();
- if (elapsed >= slowTimeout && !isSlow)
+ if (minElapsedNanos >= slowTimeoutNanos && !isSlow)
isSlow = true;
- if (elapsed >= timeout)
+ if (minElapsedNanos >= timeoutNanos)
abort();
}
}
diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
index 94260422798e..0f8555f17aa3 100644
--- a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
+++ b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
@@ -39,6 +39,8 @@
import org.apache.cassandra.utils.NoSpamLogger;
import static java.lang.System.getProperty;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
/**
* A task for monitoring in progress operations, currently only read queries, and aborting them if they time out.
@@ -68,7 +70,7 @@ class MonitoringTask
private final ScheduledFuture> reportingTask;
private final OperationsQueue failedOperationsQueue;
private final OperationsQueue slowOperationsQueue;
- private long lastLogTime;
+ private long approxLastLogTimeNanos;
@VisibleForTesting
@@ -88,10 +90,10 @@ private MonitoringTask(int reportIntervalMillis, int maxOperations)
this.failedOperationsQueue = new OperationsQueue(maxOperations);
this.slowOperationsQueue = new OperationsQueue(maxOperations);
- this.lastLogTime = ApproximateTime.currentTimeMillis();
+ this.approxLastLogTimeNanos = approxTime.now();
logger.info("Scheduling monitoring task with report interval of {} ms, max operations {}", reportIntervalMillis, maxOperations);
- this.reportingTask = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(() -> logOperations(ApproximateTime.currentTimeMillis()),
+ this.reportingTask = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(() -> logOperations(approxTime.now()),
reportIntervalMillis,
reportIntervalMillis,
TimeUnit.MILLISECONDS);
@@ -102,14 +104,14 @@ public void cancel()
reportingTask.cancel(false);
}
- static void addFailedOperation(Monitorable operation, long now)
+ static void addFailedOperation(Monitorable operation, long nowNanos)
{
- instance.failedOperationsQueue.offer(new FailedOperation(operation, now));
+ instance.failedOperationsQueue.offer(new FailedOperation(operation, nowNanos));
}
- static void addSlowOperation(Monitorable operation, long now)
+ static void addSlowOperation(Monitorable operation, long nowNanos)
{
- instance.slowOperationsQueue.offer(new SlowOperation(operation, now));
+ instance.slowOperationsQueue.offer(new SlowOperation(operation, nowNanos));
}
@VisibleForTesting
@@ -131,27 +133,27 @@ private List getLogMessages(AggregatedOperations operations)
}
@VisibleForTesting
- private void logOperations(long now)
+ private void logOperations(long approxCurrentTimeNanos)
{
- logSlowOperations(now);
- logFailedOperations(now);
+ logSlowOperations(approxCurrentTimeNanos);
+ logFailedOperations(approxCurrentTimeNanos);
- lastLogTime = now;
+ approxLastLogTimeNanos = approxCurrentTimeNanos;
}
@VisibleForTesting
- boolean logFailedOperations(long now)
+ boolean logFailedOperations(long nowNanos)
{
AggregatedOperations failedOperations = failedOperationsQueue.popOperations();
if (!failedOperations.isEmpty())
{
- long elapsed = now - lastLogTime;
+ long elapsedNanos = nowNanos - approxLastLogTimeNanos;
noSpamLogger.warn("Some operations timed out, details available at debug level (debug.log)");
if (logger.isDebugEnabled())
logger.debug("{} operations timed out in the last {} msecs:{}{}",
failedOperations.num(),
- elapsed,
+ NANOSECONDS.toMillis(elapsedNanos),
LINE_SEPARATOR,
failedOperations.getLogMessage());
return true;
@@ -161,18 +163,18 @@ boolean logFailedOperations(long now)
}
@VisibleForTesting
- boolean logSlowOperations(long now)
+ boolean logSlowOperations(long approxCurrentTimeNanos)
{
AggregatedOperations slowOperations = slowOperationsQueue.popOperations();
if (!slowOperations.isEmpty())
{
- long elapsed = now - lastLogTime;
+ long approxElapsedNanos = approxCurrentTimeNanos - approxLastLogTimeNanos;
noSpamLogger.info("Some operations were slow, details available at debug level (debug.log)");
if (logger.isDebugEnabled())
logger.debug("{} operations were slow in the last {} msecs:{}{}",
slowOperations.num(),
- elapsed,
+ NANOSECONDS.toMillis(approxElapsedNanos),
LINE_SEPARATOR,
slowOperations.getLogMessage());
return true;
@@ -314,7 +316,7 @@ protected abstract static class Operation
int numTimesReported;
/** The total time spent by this operation */
- long totalTime;
+ long totalTimeNanos;
/** The maximum time spent by this operation */
long maxTime;
@@ -326,13 +328,13 @@ protected abstract static class Operation
* this is set lazily as it takes time to build the query CQL */
private String name;
- Operation(Monitorable operation, long failedAt)
+ Operation(Monitorable operation, long failedAtNanos)
{
this.operation = operation;
numTimesReported = 1;
- totalTime = failedAt - operation.constructionTime();
- minTime = totalTime;
- maxTime = totalTime;
+ totalTimeNanos = failedAtNanos - operation.creationTimeNanos();
+ minTime = totalTimeNanos;
+ maxTime = totalTimeNanos;
}
public String name()
@@ -345,7 +347,7 @@ public String name()
void add(Operation operation)
{
numTimesReported++;
- totalTime += operation.totalTime;
+ totalTimeNanos += operation.totalTimeNanos;
maxTime = Math.max(maxTime, operation.maxTime);
minTime = Math.min(minTime, operation.minTime);
}
@@ -358,9 +360,9 @@ void add(Operation operation)
*/
private final static class FailedOperation extends Operation
{
- FailedOperation(Monitorable operation, long failedAt)
+ FailedOperation(Monitorable operation, long failedAtNanos)
{
- super(operation, failedAt);
+ super(operation, failedAtNanos);
}
public String getLogMessage()
@@ -368,17 +370,17 @@ public String getLogMessage()
if (numTimesReported == 1)
return String.format("<%s>, total time %d msec, timeout %d %s",
name(),
- totalTime,
- operation.timeout(),
+ NANOSECONDS.toMillis(totalTimeNanos),
+ NANOSECONDS.toMillis(operation.timeoutNanos()),
operation.isCrossNode() ? "msec/cross-node" : "msec");
else
return String.format("<%s> timed out %d times, avg/min/max %d/%d/%d msec, timeout %d %s",
name(),
numTimesReported,
- totalTime / numTimesReported,
- minTime,
- maxTime,
- operation.timeout(),
+ NANOSECONDS.toMillis(totalTimeNanos / numTimesReported),
+ NANOSECONDS.toMillis(minTime),
+ NANOSECONDS.toMillis(maxTime),
+ NANOSECONDS.toMillis(operation.timeoutNanos()),
operation.isCrossNode() ? "msec/cross-node" : "msec");
}
}
@@ -398,17 +400,17 @@ public String getLogMessage()
if (numTimesReported == 1)
return String.format("<%s>, time %d msec - slow timeout %d %s",
name(),
- totalTime,
- operation.slowTimeout(),
+ NANOSECONDS.toMillis(totalTimeNanos),
+ NANOSECONDS.toMillis(operation.slowTimeoutNanos()),
operation.isCrossNode() ? "msec/cross-node" : "msec");
else
return String.format("<%s>, was slow %d times: avg/min/max %d/%d/%d msec - slow timeout %d %s",
name(),
numTimesReported,
- totalTime / numTimesReported,
- minTime,
- maxTime,
- operation.slowTimeout(),
+ NANOSECONDS.toMillis(totalTimeNanos/ numTimesReported),
+ NANOSECONDS.toMillis(minTime),
+ NANOSECONDS.toMillis(maxTime),
+ NANOSECONDS.toMillis(operation.slowTimeoutNanos()),
operation.isCrossNode() ? "msec/cross-node" : "msec");
}
}
diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java b/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
index bed0958b8e2f..74a61d6feadd 100644
--- a/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
@@ -20,6 +20,7 @@
import java.util.*;
import org.apache.cassandra.db.EmptyIterators;
+import org.apache.cassandra.db.transform.FilteredPartitions;
import org.apache.cassandra.db.transform.MorePartitions;
import org.apache.cassandra.db.transform.Transformation;
import org.apache.cassandra.utils.AbstractIterator;
@@ -66,7 +67,7 @@ public static PartitionIterator concat(final List iterators)
class Extend implements MorePartitions
{
- int i = 1;
+ int i = 0;
public PartitionIterator moreContents()
{
if (i >= iterators.size())
@@ -74,7 +75,8 @@ public PartitionIterator moreContents()
return iterators.get(i++);
}
}
- return MorePartitions.extend(iterators.get(0), new Extend());
+
+ return MorePartitions.extend(EmptyIterators.partition(), new Extend());
}
public static PartitionIterator singletonIterator(RowIterator iterator)
diff --git a/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java b/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
index fac164d006a5..85d262566290 100644
--- a/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
+++ b/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
@@ -349,7 +349,7 @@ public ListenableFuture run()
List> tasks = new ArrayList<>(tables.size());
for (ColumnFamilyStore cfs : tables)
{
- cfs.forceBlockingFlush();
+ cfs.forceBlockingFlushToSSTable();
ListenableFutureTask task = ListenableFutureTask.create(getAcquisitionCallable(cfs, tokenRanges.ranges(), prsId, acquireRetrySeconds, acquireSleepMillis));
executor.submit(task);
tasks.add(task);
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
index eb993ff0f8d1..c362d1174329 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
@@ -99,8 +99,9 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException
// when compressed, report total bytes of compressed chunks read since remoteFile.size is the sum of chunks transferred
session.progress(filename, ProgressInfo.Direction.IN, cis.getTotalCompressedBytesRead(), totalSize);
}
+ assert in.getBytesRead() == sectionLength;
}
- logger.debug("[Stream #{}] Finished receiving file #{} from {} readBytes = {}, totalSize = {}", session.planId(), fileSeqNum,
+ logger.trace("[Stream #{}] Finished receiving file #{} from {} readBytes = {}, totalSize = {}", session.planId(), fileSeqNum,
session.peer, FBUtilities.prettyPrintMemory(cis.getTotalCompressedBytesRead()), FBUtilities.prettyPrintMemory(totalSize));
return writer;
}
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java
index 3b971f885942..efbccdcf25b3 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java
@@ -31,8 +31,7 @@
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.util.ChannelProxy;
import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
import org.apache.cassandra.streaming.ProgressInfo;
import org.apache.cassandra.streaming.StreamSession;
import org.apache.cassandra.utils.FBUtilities;
@@ -55,10 +54,9 @@ public CassandraCompressedStreamWriter(SSTableReader sstable, Collection {
+ ByteBuffer outBuffer = bufferSupplier.get(toTransfer);
+ long read = fc.read(outBuffer, position);
+ assert read == toTransfer : String.format("could not read required number of bytes from file to be streamed: read %d bytes, wanted %d bytes", read, toTransfer);
outBuffer.flip();
- output.writeToChannel(outBuffer);
- }
- catch (IOException e)
- {
- FileUtils.clean(outBuffer);
- throw e;
- }
-
- bytesTransferred += lastWrite;
- progress += lastWrite;
+ }, limiter);
+
+ bytesTransferred += toTransfer;
+ progress += toTransfer;
session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize);
}
}
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java
index 7a20110d0bf3..401b20ed56ad 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java
@@ -27,7 +27,7 @@
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
import org.apache.cassandra.streaming.ProgressInfo;
import org.apache.cassandra.streaming.StreamManager;
import org.apache.cassandra.streaming.StreamSession;
@@ -58,11 +58,11 @@ public CassandraEntireSSTableStreamWriter(SSTableReader sstable, StreamSession s
/**
* Stream the entire file to given channel.
*