This is an automated email from the ASF dual-hosted git repository. dcapwell pushed a commit to branch cassandra-4.0 in repository https://gitbox.apache.org/repos/asf/cassandra.git
commit d69ea282162d4c69c8f75b352c5e0f863fc1b24e Merge: 658f3e1 54f7308 Author: David Capwell <dcapw...@apache.org> AuthorDate: Fri May 21 16:53:28 2021 -0700 Merge branch 'cassandra-3.11' into cassandra-4.0 CHANGES.txt | 1 + .../org/apache/cassandra/db/compaction/Verifier.java | 12 ++++++------ .../org/apache/cassandra/tools/StandaloneVerifier.java | 17 ++++------------- src/java/org/apache/cassandra/utils/OutputHandler.java | 4 ++++ 4 files changed, 15 insertions(+), 19 deletions(-) diff --cc CHANGES.txt index e84e3fe,2e419c9..e2305d1 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -14,7 -2,12 +14,8 @@@ Merged from 3.11 * Maps $CASSANDRA_LOG_DIR to cassandra.logdir java property when executing nodetool (CASSANDRA-16199) * Nodetool garbagecollect should retain SSTableLevel for LCS (CASSANDRA-16634) * Ignore stale acks received in the shadow round (CASSANDRA-16588) - * Add autocomplete and error messages for provide_overlapping_tombstones (CASSANDRA-16350) - * Add StorageServiceMBean.getKeyspaceReplicationInfo(keyspaceName) (CASSANDRA-16447) - * Make sure sstables with moved starts are removed correctly in LeveledGenerations (CASSANDRA-16552) - * Upgrade jackson-databind to 2.9.10.8 (CASSANDRA-16462) Merged from 3.0: + * StandaloneVerifier does not fail when unable to verify SSTables, it only fails if Corruption is thrown (CASSANDRA-16683) * Fix bloom filter false ratio calculation by including true negatives (CASSANDRA-15834) * Prevent loss of commit log data when moving sstables between nodes (CASSANDRA-16619) * Fix materialized view builders inserting truncated data (CASSANDRA-16567) diff --cc src/java/org/apache/cassandra/db/compaction/Verifier.java index 18b415c,8c5e8bb..30e74ad --- a/src/java/org/apache/cassandra/db/compaction/Verifier.java +++ b/src/java/org/apache/cassandra/db/compaction/Verifier.java @@@ -143,72 -102,13 +143,72 @@@ public class Verifier implements Closea } catch (Throwable t) { - outputHandler.warn(t.getMessage()); - outputHandler.debug(t.getMessage()); - markAndThrow(false); ++ outputHandler.warn(t); + markAndThrow(t, false); } - outputHandler.output(String.format("Checking computed hash of %s ", sstable)); + try + { + outputHandler.debug("Deserializing index for "+sstable); + deserializeIndex(sstable); + } + catch (Throwable t) + { - outputHandler.warn(t.getMessage()); ++ outputHandler.warn(t); + markAndThrow(t); + } + + try + { + outputHandler.debug("Deserializing index summary for "+sstable); + deserializeIndexSummary(sstable); + } + catch (Throwable t) + { + outputHandler.output("Index summary is corrupt - if it is removed it will get rebuilt on startup "+sstable.descriptor.filenameFor(Component.SUMMARY)); - outputHandler.warn(t.getMessage()); ++ outputHandler.warn(t); + markAndThrow(t, false); + } + + try + { + outputHandler.debug("Deserializing bloom filter for "+sstable); + deserializeBloomFilter(sstable); + + } + catch (Throwable t) + { - outputHandler.warn(t.getMessage()); ++ outputHandler.warn(t); + markAndThrow(t); + } + + if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner)) + { + outputHandler.debug("Checking that all tokens are owned by the current node"); + try (KeyIterator iter = new KeyIterator(sstable.descriptor, sstable.metadata())) + { + List<Range<Token>> ownedRanges = Range.normalize(tokenLookup.apply(cfs.metadata.keyspace)); + if (ownedRanges.isEmpty()) + return; + RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(ownedRanges); + while (iter.hasNext()) + { + DecoratedKey key = iter.next(); + rangeOwnHelper.validate(key); + } + } + catch (Throwable t) + { - outputHandler.warn(t.getMessage()); ++ outputHandler.warn(t); + markAndThrow(t); + } + } + + if (options.quick) + return; // Verify will use the Digest files, which works for both compressed and uncompressed sstables + outputHandler.output(String.format("Checking computed hash of %s ", sstable)); try { validator = null; @@@ -226,8 -127,8 +226,8 @@@ } catch (IOException e) { - outputHandler.warn(e.getMessage()); - outputHandler.debug(e.getMessage()); - markAndThrow(); ++ outputHandler.warn(e); + markAndThrow(e); } finally { diff --cc src/java/org/apache/cassandra/tools/StandaloneVerifier.java index 9074418,5690553..4e4a80a --- a/src/java/org/apache/cassandra/tools/StandaloneVerifier.java +++ b/src/java/org/apache/cassandra/tools/StandaloneVerifier.java @@@ -100,28 -92,12 +100,19 @@@ public class StandaloneVerifie e.printStackTrace(System.err); } } - + Verifier.Options verifyOptions = Verifier.options().invokeDiskFailurePolicy(false) + .extendedVerification(options.extended) + .checkVersion(options.checkVersion) + .mutateRepairStatus(options.mutateRepairStatus) + .checkOwnsTokens(!options.tokens.isEmpty()) + .tokenLookup(ignore -> options.tokens) + .build(); + handler.output("Running verifier with the following options: " + verifyOptions); for (SSTableReader sstable : sstables) { - try - try (Verifier verifier = new Verifier(cfs, sstable, handler, true)) ++ try (Verifier verifier = new Verifier(cfs, sstable, handler, true, verifyOptions)) { - - try (Verifier verifier = new Verifier(cfs, sstable, handler, true, verifyOptions)) - { - verifier.verify(); - } - catch (CorruptSSTableException cs) - { - System.err.println(String.format("Error verifying %s: %s", sstable, cs.getMessage())); - hasFailed = true; - } - verifier.verify(extended); ++ verifier.verify(); } catch (Exception e) { diff --cc src/java/org/apache/cassandra/utils/OutputHandler.java index 3b10d61,88c54ce..820160d --- a/src/java/org/apache/cassandra/utils/OutputHandler.java +++ b/src/java/org/apache/cassandra/utils/OutputHandler.java @@@ -34,6 -32,6 +34,10 @@@ public interface OutputHandle // called when the user needs to be warn public void warn(String msg); public void warn(String msg, Throwable th); ++ public default void warn(Throwable th) ++ { ++ warn(th.getMessage(), th); ++ } public static class LogOutput implements OutputHandler { --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org