You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2022/02/16 10:21:49 UTC

[cassandra] branch trunk updated (81922c5 -> e56bf10)

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

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


    from 81922c5  Intoduce Harry to the tree
     new 85c202d  Lazy transaction log replica creation allows incorrect replica content divergence during anticompaction
     new eeec360  Merge branch 'cassandra-3.0' into cassandra-3.11
     new 2687cf1  Merge branch 'cassandra-3.11' into cassandra-4.0
     new e56bf10  Merge branch 'cassandra-4.0' into trunk

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 CHANGES.txt                                        |  8 +-
 .../org/apache/cassandra/db/lifecycle/LogFile.java | 22 +++---
 .../cassandra/db/lifecycle/LogTransactionTest.java | 91 +++++++++++++++-------
 3 files changed, 82 insertions(+), 39 deletions(-)

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


[cassandra] 01/01: Merge branch 'cassandra-4.0' into trunk

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e56bf105a161b1d9677ea4c9c70436f3169edb9c
Merge: 81922c5 2687cf1
Author: Marcus Eriksson <ma...@apache.org>
AuthorDate: Wed Feb 16 11:09:55 2022 +0100

    Merge branch 'cassandra-4.0' into trunk

 CHANGES.txt                                        |  8 +-
 .../org/apache/cassandra/db/lifecycle/LogFile.java | 22 +++---
 .../cassandra/db/lifecycle/LogTransactionTest.java | 91 +++++++++++++++-------
 3 files changed, 82 insertions(+), 39 deletions(-)

diff --cc CHANGES.txt
index 51e39bf,c64b602..9043898
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,95 -1,9 +1,101 @@@
 +4.1
 + * 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:
++
+ 4.0.4
+  * Lazy transaction log replica creation allows incorrect replica content divergence during anticompaction (CASSANDRA-17273)
+ 
+ 4.0.3
+  * Deprecate otc_coalescing_strategy, otc_coalescing_window_us, otc_coalescing_enough_coalesced_messages,
+    otc_backlog_expiration_interval_ms (CASSANDRA-17377)
   * Improve start up processing of Incremental Repair information read from system.repairs (CASSANDRA-17342)
  
  4.0.2
diff --cc src/java/org/apache/cassandra/db/lifecycle/LogFile.java
index 5a468f8,9053034..767cb1c
--- a/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
@@@ -346,11 -340,15 +341,15 @@@ final class LogFile implements AutoClos
      private LogRecord makeRecord(Type type, SSTable table, LogRecord record)
      {
          assert type == Type.ADD || type == Type.REMOVE;
+         maybeCreateReplica(table);
+         return record.asType(type);
+     }
  
-         File directory = table.descriptor.directory;
+     private void maybeCreateReplica(SSTable sstable)
+     {
+         File directory = sstable.descriptor.directory;
 -        String fileName = StringUtils.join(directory, File.separator, getFileName());
 +        String fileName = StringUtils.join(directory, File.pathSeparator(), getFileName());
-         replicas.maybeCreateReplica(directory, fileName, records);
-         return record.asType(type);
+         replicas.maybeCreateReplica(directory, fileName, onDiskRecords);
      }
  
      void addRecord(LogRecord record)
diff --cc test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
index 3ea710d,a4e74ce..133a51d
--- a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
@@@ -17,10 -17,18 +17,16 @@@
   */
  package org.apache.cassandra.db.lifecycle;
  
 -import java.io.File;
 -import java.io.IOError;
 +
  import java.io.IOException;
 -import java.io.RandomAccessFile;
  import java.nio.file.Files;
- import java.util.*;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
  import java.util.function.BiConsumer;
  import java.util.function.Consumer;
  import java.util.stream.Collectors;
@@@ -28,13 -36,9 +34,10 @@@
  import com.google.common.collect.ImmutableSet;
  import com.google.common.collect.Iterables;
  import com.google.common.collect.Sets;
- 
- import org.apache.cassandra.Util;
- import org.apache.cassandra.io.util.File;
  import org.junit.BeforeClass;
  import org.junit.Test;
- import org.junit.Assert;
  
++import org.apache.cassandra.Util;
  import org.apache.cassandra.db.ColumnFamilyStore;
  import org.apache.cassandra.db.Directories;
  import org.apache.cassandra.db.SerializationHeader;
@@@ -45,6 -50,6 +49,7 @@@ import org.apache.cassandra.io.sstable.
  import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
  import org.apache.cassandra.io.sstable.metadata.MetadataType;
  import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
++import org.apache.cassandra.io.util.File;
  import org.apache.cassandra.io.util.FileHandle;
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.schema.MockSchema;
@@@ -223,6 -229,37 +229,37 @@@ public class LogTransactionTest extend
      }
  
      @Test
+     public void testUntrackIdenticalLogFilesOnDisk() throws Throwable
+     {
+         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
 -        File datadir1 = Files.createTempDirectory("datadir1").toFile();
 -        File datadir2 = Files.createTempDirectory("datadir2").toFile();
++        File datadir1 = new File(Files.createTempDirectory("datadir1"));
++        File datadir2 = new File(Files.createTempDirectory("datadir2"));
+         SSTableReader sstable1 = sstable(datadir1, cfs, 1, 128);
+         SSTableReader sstable2 = sstable(datadir2, cfs, 1, 128);
+ 
+ 
+         for (Consumer<LogTransaction> c : Arrays.<Consumer<LogTransaction>>asList((log) -> log.trackNew(sstable2),
+                                                                                   (log) -> log.obsoleted(sstable2),
+                                                                                   (log) -> log.txnFile().addAll(LogRecord.Type.ADD, Collections.singleton(sstable2))))
+         {
+             try (LogTransaction log = new LogTransaction(OperationType.COMPACTION))
+             {
+                 log.trackNew(sstable1); // creates a log file in datadir1
+                 log.untrackNew(sstable1); // removes sstable1 from `records`, but still on disk & in `onDiskRecords`
+ 
+                 c.accept(log);  // creates a log file in datadir2, based on contents in onDiskRecords
+                 byte[] log1 = Files.readAllBytes(log.logFiles().get(0).toPath());
+                 byte[] log2 = Files.readAllBytes(log.logFiles().get(1).toPath());
+                 assertArrayEquals(log1, log2);
+             }
+         }
+         sstable1.selfRef().release();
+         sstable2.selfRef().release();
+         Thread.sleep(1);
+         LogTransaction.waitForDeletions();
+     }
+ 
+     @Test
      public void testCommitSameDesc() throws Throwable
      {
          ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);

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