This is an automated email from the ASF dual-hosted git repository.

ycai pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit df9586da3323bcd817b658ed7cab8dc94a66c20d
Merge: c408931 fd5e2ad
Author: Yifan Cai <y...@apache.org>
AuthorDate: Fri Mar 4 10:43:49 2022 -0800

    Merge branch 'cassandra-4.0' into trunk

 CHANGES.txt                                        |  1 +
 .../cassandra/repair/consistent/LocalSessions.java | 24 ++++++++++++++++++----
 2 files changed, 21 insertions(+), 4 deletions(-)

diff --cc CHANGES.txt
index f08ccaa,d9d4a77..206ab5b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,100 -1,5 +1,101 @@@
 -4.0.4
 +4.1
 + * Expose streaming as a vtable (CASSANDRA-17390)
 + * Make startup checks configurable (CASSANDRA-17220)
 + * Add guardrail for number of partition keys on IN queries (CASSANDRA-17186)
 + * update Python test framework from nose to pytest (CASSANDRA-17293)
 + * Fix improper CDC commit log segments deletion in non-blocking mode 
(CASSANDRA-17233)
 + * Add support for string concatenations through the + operator 
(CASSANDRA-17190)
 + * Limit the maximum hints size per host (CASSANDRA-17142)
 + * Add a virtual table for exposing batch metrics (CASSANDRA-17225)
 + * Flatten guardrails config (CASSANDRA-17353)
 + * Instance failed to start up due to NPE in 
StartupClusterConnectivityChecker (CASSANDRA-17347)
 + * add the shorter version of version flag (-v) in cqlsh (CASSANDRA-17236)
 + * Make vtables accessible via internode messaging (CASSANDRA-17295)
 + * Add support for PEM based key material for SSL (CASSANDRA-17031)
 + * Standardize storage configuration parameters' names. Support unit 
suffixes. (CASSANDRA-15234)
 + * Remove support for Windows (CASSANDRA-16956)
 + * Runtime-configurable YAML option to prohibit USE statements 
(CASSANDRA-17318)
 + * When streaming sees a ClosedChannelException this triggers the disk 
failure policy (CASSANDRA-17116)
 + * Add a virtual table for exposing prepared statements metrics 
(CASSANDRA-17224)
 + * Remove python 2.x support from cqlsh (CASSANDRA-17242)
 + * Prewarm role and credential caches to avoid timeouts at startup 
(CASSANDRA-16958)
 + * Make capacity/validity/updateinterval/activeupdate for Auth Caches 
configurable via nodetool (CASSANDRA-17063)
 + * Added startup check for read_ahead_kb setting (CASSANDRA-16436)
 + * Avoid unecessary array allocations and initializations when performing 
query checks (CASSANDRA-17209)
 + * Add guardrail for list operations that require read before write 
(CASSANDRA-17154)
 + * Migrate thresholds for number of keyspaces and tables to guardrails 
(CASSANDRA-17195)
 + * Remove self-reference in SSTableTidier (CASSANDRA-17205)
 + * Add guardrail for query page size (CASSANDRA-17189)
 + * Allow column_index_size_in_kb to be configurable through nodetool 
(CASSANDRA-17121)
 + * Emit a metric for number of local read and write calls
 + * Add non-blocking mode for CDC writes (CASSANDRA-17001)
 + * Add guardrails framework (CASSANDRA-17147)
 + * Harden resource management on SSTable components to prevent future leaks 
(CASSANDRA-17174)
 + * Make nodes more resilient to local unrelated files during startup 
(CASSANDRA-17082)
 + * repair prepare message would produce a wrong error message if network 
timeout happened rather than reply wait timeout (CASSANDRA-16992)
 + * Log queries that fail on timeout or unavailable errors up to once per 
minute by default (CASSANDRA-17159)
 + * Refactor normal/preview/IR repair to standardize repair cleanup and error 
handling of failed RepairJobs (CASSANDRA-17069)
 + * Log missing peers in StartupClusterConnectivityChecker (CASSANDRA-17130)
 + * Introduce separate rate limiting settings for entire SSTable streaming 
(CASSANDRA-17065)
 + * Implement Virtual Tables for Auth Caches (CASSANDRA-16914)
 + * Actively update auth cache in the background (CASSANDRA-16957)
 + * Add unix time conversion functions (CASSANDRA-17029)
 + * JVMStabilityInspector.forceHeapSpaceOomMaybe should handle all non-heap 
OOMs rather than only supporting direct only (CASSANDRA-17128)
 + * Forbid other Future implementations with checkstyle (CASSANDRA-17055)
 + * commit log was switched from non-daemon to daemon threads, which causes 
the JVM to exit in some case as no non-daemon threads are active 
(CASSANDRA-17085)
 + * Add a Denylist to block reads and writes on specific partition keys 
(CASSANDRA-12106)
 + * v4+ protocol did not clean up client warnings, which caused leaking the 
state (CASSANDRA-17054)
 + * Remove duplicate toCQLString in ReadCommand (CASSANDRA-17023)
 + * Ensure hint window is persistent across restarts of a node 
(CASSANDRA-14309)
 + * Allow to GRANT or REVOKE multiple permissions in a single statement 
(CASSANDRA-17030)
 + * Allow to grant permission for all tables in a keyspace (CASSANDRA-17027)
 + * Log time spent writing keys during compaction (CASSANDRA-17037)
 + * Make nodetool compactionstats and sstable_tasks consistent 
(CASSANDRA-16976)
 + * Add metrics and logging around index summary redistribution 
(CASSANDRA-17036)
 + * Add configuration options for minimum allowable replication factor and 
default replication factor (CASSANDRA-14557)
 + * Expose information about stored hints via a nodetool command and a virtual 
table (CASSANDRA-14795)
 + * Add broadcast_rpc_address to system.local (CASSANDRA-11181)
 + * Add support for type casting in WHERE clause components and in the values 
of INSERT/UPDATE statements (CASSANDRA-14337)
 + * add credentials file support to CQLSH (CASSANDRA-16983)
 + * Skip remaining bytes in the Envelope buffer when a ProtocolException is 
thrown to avoid double decoding (CASSANDRA-17026)
 + * Allow reverse iteration of resources during permissions checking 
(CASSANDRA-17016)
 + * Add feature to verify correct ownership of attached locations on disk at 
startup (CASSANDRA-16879)
 + * Make SSLContext creation pluggable/extensible (CASSANDRA-16666)
 + * Add soft/hard limits to local reads to protect against reading too much 
data in a single query (CASSANDRA-16896)
 + * Avoid token cache invalidation for removing a non-member node 
(CASSANDRA-15290)
 + * Allow configuration of consistency levels on auth operations 
(CASSANDRA-12988)
 + * Add number of sstables in a compaction to compactionstats output 
(CASSANDRA-16844)
 + * Upgrade Caffeine to 2.9.2 (CASSANDRA-15153)
 + * Allow DELETE and TRUNCATE to work on Virtual Tables if the implementation 
allows it (CASSANDRA-16806)
 + * Include SASI components to snapshots (CASSANDRA-15134)
 + * Fix missed wait latencies in the output of `nodetool tpstats -F` 
(CASSANDRA-16938)
 + * Reduce native transport max frame size to 16MB (CASSANDRA-16886)
 + * Add support for filtering using IN restrictions (CASSANDRA-14344)
 + * Provide a nodetool command to invalidate auth caches (CASSANDRA-16404)
 + * Catch read repair timeout exceptions and add metric (CASSANDRA-16880)
 + * Exclude Jackson 1.x transitive dependency of hadoop* provided dependencies 
(CASSANDRA-16854)
 + * Add client warnings and abort to tombstone and coordinator reads which go 
past a low/high watermark (CASSANDRA-16850)
 + * Add TTL support to nodetool snapshots (CASSANDRA-16789)
 + * Allow CommitLogSegmentReader to optionally skip sync marker CRC checks 
(CASSANDRA-16842)
 + * allow blocking IPs from updating metrics about traffic (CASSANDRA-16859)
 + * Request-Based Native Transport Rate-Limiting (CASSANDRA-16663)
 + * Implement nodetool getauditlog command (CASSANDRA-16725)
 + * Clean up repair code (CASSANDRA-13720)
 + * Background schedule to clean up orphaned hints files (CASSANDRA-16815)
 + * Modify SecondaryIndexManager#indexPartition() to retrieve only columns for 
which indexes are actually being built (CASSANDRA-16776)
 + * Batch the token metadata update to improve the speed (CASSANDRA-15291)
 + * Reduce the log level on "expected" repair exceptions (CASSANDRA-16775)
 + * Make JMXTimer expose attributes using consistent time unit 
(CASSANDRA-16760)
 + * Remove check on gossip status from DynamicEndpointSnitch::updateScores 
(CASSANDRA-11671)
 + * Fix AbstractReadQuery::toCQLString not returning valid CQL 
(CASSANDRA-16510)
 + * Log when compacting many tombstones (CASSANDRA-16780)
 + * Display bytes per level in tablestats for LCS tables (CASSANDRA-16799)
 + * Add isolated flush timer to CommitLogMetrics and ensure writes correspond 
to single WaitingOnCommit data points (CASSANDRA-16701)
 + * Add a system property to set hostId if not yet initialized 
(CASSANDRA-14582)
 + * GossiperTest.testHasVersion3Nodes didn't take into account trunk version 
changes, fixed to rely on latest version (CASSANDRA-16651)
 + * Update JNA library to 5.9.0 and snappy-java to version 1.1.8.4 
(CASSANDRA-17040)
 +Merged from 4.0:
+  * Fix race condition bug during local session repair (CASSANDRA-17335)
   * Fix ignored streaming encryption settings in sstableloader 
(CASSANDRA-17367)
   * Streaming tasks handle empty SSTables correctly (CASSANDRA-16349)
   * Prevent SSTableLoader from doing unnecessary work (CASSANDRA-16349)
diff --cc src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
index f6b4423,6f7b93e..29a744d
--- a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
+++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
@@@ -829,11 -828,9 +829,8 @@@ public class LocalSession
                  try
                  {
                      logger.info("Prepare phase for incremental repair session 
{} completed", sessionID);
-                     if (session.getState() != FAILED)
-                         setStateAndSave(session, PREPARED);
-                     else
+                     if (!prepareSessionExceptFailed(session))
                          logger.info("Session {} failed before anticompaction 
completed", sessionID);
--
                      Message<PrepareConsistentResponse> message =
                          Message.out(PREPARE_CONSISTENT_RSP,
                                      new PrepareConsistentResponse(sessionID, 
getBroadcastAddressAndPort(), session.getState() != FAILED));
@@@ -865,9 -857,28 +862,28 @@@
                      executor.shutdown();
                  }
              }
 -        }, MoreExecutors.directExecutor());
 +        });
      }
  
+     /**
+      * Checks for the session state, and sets it to prepared unless it is on 
a failed state.
+      * Making the checks inside a synchronized block to prevent the session 
state from
+      * being changed between the read and the update.
+      *
+      * @param session The local session to be set to prepared.
+      * @return true if the session is prepared, false if not, i.e. session 
failed
+      */
+     private boolean prepareSessionExceptFailed(LocalSession session) {
+         synchronized (session)
+         {
+             if (session.getState() == FAILED)
+                 return false;
+ 
+             setStateAndSave(session, PREPARED);
+             return true;
+         }
+     }
+ 
      public void maybeSetRepairing(UUID sessionID)
      {
          LocalSession session = getSession(sessionID);

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to