public class CompactionManager extends java.lang.Object implements CompactionManagerMBean
A singleton which manages a private executor of ongoing compactions.
Scheduling for compaction is accomplished by swapping sstables to be compacted into a set via Tracker. New scheduling attempts will ignore currently compacting sstables.Modifier and Type | Class and Description |
---|---|
static class |
CompactionManager.AllSSTableOpStatus |
static interface |
CompactionManager.CompactionPauser |
static class |
CompactionManager.ValidationExecutor |
Modifier and Type | Field and Description |
---|---|
ActiveCompactions |
active |
java.util.concurrent.atomic.AtomicInteger |
currentlyBackgroundUpgrading |
static int |
GC_ALL |
static CompactionManager |
instance |
static io.netty.util.concurrent.FastThreadLocal<java.lang.Boolean> |
isCompactionManager |
static java.lang.String |
MBEAN_OBJECT_NAME |
static int |
NO_GC |
Constructor and Description |
---|
CompactionManager() |
Modifier and Type | Method and Description |
---|---|
static SSTableWriter |
createWriter(ColumnFamilyStore cfs,
java.io.File compactionFileLocation,
long expectedBloomFilterSize,
long repairedAt,
java.util.UUID pendingRepair,
boolean isTransient,
SSTableReader sstable,
LifecycleTransaction txn) |
static SSTableWriter |
createWriterForAntiCompaction(ColumnFamilyStore cfs,
java.io.File compactionFileLocation,
int expectedBloomFilterSize,
long repairedAt,
java.util.UUID pendingRepair,
boolean isTransient,
java.util.Collection<SSTableReader> sstables,
ILifecycleTransaction txn) |
void |
disableAutoCompaction() |
void |
finishCompactionsAndShutdown(long timeout,
java.util.concurrent.TimeUnit unit) |
void |
forceCompactionForTokenRange(ColumnFamilyStore cfStore,
java.util.Collection<Range<Token>> ranges) |
void |
forceShutdown()
Shutdowns both compaction and validation executors, cancels running compaction / validation,
and waits for tasks to complete if tasks were not cancelable.
|
void |
forceUserDefinedCleanup(java.lang.String dataFiles)
Triggers the cleanup of user specified sstables.
|
void |
forceUserDefinedCompaction(java.lang.String dataFiles)
Triggers the compaction of user specified sstables.
|
int |
getActiveCompactions() |
static CompactionIterator |
getAntiCompactionIterator(java.util.List<ISSTableScanner> scanners,
CompactionController controller,
int nowInSec,
java.util.UUID timeUUID,
ActiveCompactionsTracker activeCompactions,
java.util.function.BooleanSupplier isCancelled) |
boolean |
getAutomaticSSTableUpgradeEnabled()
Get automatic sstable upgrade enabled
|
org.apache.cassandra.db.compaction.CompactionManager.BackgroundCompactionCandidate |
getBackgroundCompactionCandidate(ColumnFamilyStore cfs) |
javax.management.openmbean.TabularData |
getCompactionHistory()
compaction history
|
java.util.List<java.util.Map<java.lang.String,java.lang.String>> |
getCompactions()
List of running compaction objects.
|
java.util.List<java.lang.String> |
getCompactionSummary()
List of running compaction summary strings.
|
long |
getCompletedTasks() |
int |
getCoreCompactorThreads()
Returns core size of compaction thread pool
|
int |
getCoreValidationThreads()
Returns core size of validation thread pool
|
int |
getCoreViewBuildThreads()
Returns core size of view build thread pool
|
static int |
getDefaultGcBefore(ColumnFamilyStore cfs,
int nowInSec) |
boolean |
getDisableSTCSInL0()
Enable / disable STCS in L0
|
int |
getMaxConcurrentAutoUpgradeTasks()
Get the number of concurrent sstable upgrade tasks we should run
when automatic sstable upgrades are enabled
|
int |
getMaximumCompactorThreads()
Returns maximum size of compaction thread pool
|
int |
getMaximumValidatorThreads()
Returns size of validator thread pool
|
int |
getMaximumViewBuildThreads()
Returns size of view build thread pool
|
CompactionMetrics |
getMetrics() |
int |
getPendingTasks() |
com.google.common.util.concurrent.RateLimiter |
getRateLimiter()
Gets compaction rate limiter.
|
java.util.List<CompactionInfo> |
getSSTableTasks() |
long |
getTotalBytesCompacted() |
long |
getTotalCompactionsCompleted() |
void |
incrementAborted() |
void |
incrementCompactionsReduced() |
void |
incrementSstablesDropppedFromCompactions(long num) |
void |
interruptCompactionFor(java.lang.Iterable<TableMetadata> columnFamilies,
java.util.function.Predicate<SSTableReader> sstablePredicate,
boolean interruptValidation)
Try to stop all of the compactions for given ColumnFamilies.
|
void |
interruptCompactionForCFs(java.lang.Iterable<ColumnFamilyStore> cfss,
java.util.function.Predicate<SSTableReader> sstablePredicate,
boolean interruptValidation) |
boolean |
isCompacting(java.lang.Iterable<ColumnFamilyStore> cfses,
java.util.function.Predicate<SSTableReader> sstablePredicate) |
boolean |
isGlobalCompactionPaused()
Return whether "global" compactions should be paused, used by ColumnFamilyStore#runWithCompactionsDisabled
a global compaction is one that includes several/all tables, currently only IndexSummaryBuilder
|
static boolean |
needsCleanup(SSTableReader sstable,
java.util.Collection<Range<Token>> ownedRanges)
Determines if a cleanup would actually remove any data in this SSTable based
on a set of owned ranges.
|
CompactionManager.CompactionPauser |
pauseGlobalCompaction() |
void |
performAnticompaction(ColumnFamilyStore cfs,
RangesAtEndpoint replicas,
Refs<SSTableReader> validatedForRepair,
LifecycleTransaction txn,
java.util.UUID sessionID,
java.util.function.BooleanSupplier isCancelled)
Make sure the {validatedForRepair} are marked for compaction before calling this.
|
CompactionManager.AllSSTableOpStatus |
performCleanup(ColumnFamilyStore cfStore,
int jobs) |
CompactionManager.AllSSTableOpStatus |
performGarbageCollection(ColumnFamilyStore cfStore,
CompactionParams.TombstoneOption tombstoneOption,
int jobs) |
void |
performMaximal(ColumnFamilyStore cfStore,
boolean splitOutput) |
CompactionManager.AllSSTableOpStatus |
performScrub(ColumnFamilyStore cfs,
boolean skipCorrupted,
boolean checkData,
boolean reinsertOverflowedTTL,
int jobs) |
CompactionManager.AllSSTableOpStatus |
performScrub(ColumnFamilyStore cfs,
boolean skipCorrupted,
boolean checkData,
int jobs) |
CompactionManager.AllSSTableOpStatus |
performSSTableRewrite(ColumnFamilyStore cfs,
boolean excludeCurrentVersion,
int jobs) |
CompactionManager.AllSSTableOpStatus |
performVerify(ColumnFamilyStore cfs,
Verifier.Options options) |
CompactionManager.AllSSTableOpStatus |
relocateSSTables(ColumnFamilyStore cfs,
int jobs) |
java.util.List<SSTableReader> |
runIndexSummaryRedistribution(IndexSummaryRedistribution redistribution) |
void |
setAutomaticSSTableUpgradeEnabled(boolean enabled)
Set if automatic sstable upgrade should be enabled
|
void |
setConcurrentCompactors(int value) |
void |
setConcurrentValidations() |
void |
setConcurrentViewBuilders(int value) |
void |
setCoreCompactorThreads(int number)
Allows user to resize maximum size of the compaction thread pool.
|
void |
setCoreValidationThreads(int number)
Allows user to resize maximum size of the compaction thread pool.
|
void |
setCoreViewBuildThreads(int number)
Allows user to resize maximum size of the view build thread pool.
|
void |
setDisableSTCSInL0(boolean disabled) |
void |
setMaxConcurrentAutoUpgradeTasks(int value)
Set the number of concurrent sstable upgrade tasks we should run
when automatic sstable upgrades are enabled
|
void |
setMaximumCompactorThreads(int number)
Allows user to resize maximum size of the compaction thread pool.
|
void |
setMaximumValidatorThreads(int number)
Allows user to resize maximum size of the validator thread pool.
|
void |
setMaximumViewBuildThreads(int number)
Allows user to resize maximum size of the view build thread pool.
|
void |
setRate(double throughPutMbPerSec)
Sets the rate for the rate limiter.
|
void |
stopCompaction(java.lang.String type)
Stop all running compaction-like tasks having the provided
type . |
void |
stopCompactionById(java.lang.String compactionId)
Stop an individual running compaction using the compactionId.
|
java.util.List<java.util.concurrent.Future<?>> |
submitBackground(ColumnFamilyStore cfs)
Call this whenever a compaction might be needed on the given columnfamily.
|
java.util.concurrent.Future<?> |
submitCacheWrite(AutoSavingCache.Writer writer) |
com.google.common.util.concurrent.ListenableFuture<?> |
submitIndexBuild(SecondaryIndexBuilder builder)
Is not scheduled, because it is performing disjoint work from sstable compaction.
|
java.util.List<java.util.concurrent.Future<?>> |
submitMaximal(ColumnFamilyStore cfStore,
int gcBefore,
boolean splitOutput) |
com.google.common.util.concurrent.ListenableFuture<?> |
submitPendingAntiCompaction(ColumnFamilyStore cfs,
RangesAtEndpoint tokenRanges,
Refs<SSTableReader> sstables,
LifecycleTransaction txn,
java.util.UUID sessionId,
java.util.function.BooleanSupplier isCancelled)
Splits the given token ranges of the given sstables into a pending repair silo
|
java.util.concurrent.Future<?> |
submitUserDefined(ColumnFamilyStore cfs,
java.util.Collection<Descriptor> dataFiles,
int gcBefore) |
java.util.concurrent.Future<?> |
submitValidation(java.util.concurrent.Callable<java.lang.Object> validation) |
com.google.common.util.concurrent.ListenableFuture<java.lang.Long> |
submitViewBuilder(ViewBuilderTask task) |
void |
waitForCessation(java.lang.Iterable<ColumnFamilyStore> cfss,
java.util.function.Predicate<SSTableReader> sstablePredicate) |
public static final java.lang.String MBEAN_OBJECT_NAME
public static final CompactionManager instance
public final java.util.concurrent.atomic.AtomicInteger currentlyBackgroundUpgrading
public static final int NO_GC
public static final int GC_ALL
public static final io.netty.util.concurrent.FastThreadLocal<java.lang.Boolean> isCompactionManager
public final ActiveCompactions active
public CompactionMetrics getMetrics()
public com.google.common.util.concurrent.RateLimiter getRateLimiter()
public void setRate(double throughPutMbPerSec)
throughPutMbPerSec
- throughput to set in mb per secondpublic java.util.List<java.util.concurrent.Future<?>> submitBackground(ColumnFamilyStore cfs)
public boolean isCompacting(java.lang.Iterable<ColumnFamilyStore> cfses, java.util.function.Predicate<SSTableReader> sstablePredicate)
public void forceShutdown()
public void finishCompactionsAndShutdown(long timeout, java.util.concurrent.TimeUnit unit) throws java.lang.InterruptedException
java.lang.InterruptedException
public org.apache.cassandra.db.compaction.CompactionManager.BackgroundCompactionCandidate getBackgroundCompactionCandidate(ColumnFamilyStore cfs)
public CompactionManager.AllSSTableOpStatus performScrub(ColumnFamilyStore cfs, boolean skipCorrupted, boolean checkData, int jobs) throws java.lang.InterruptedException, java.util.concurrent.ExecutionException
java.lang.InterruptedException
java.util.concurrent.ExecutionException
public CompactionManager.AllSSTableOpStatus performScrub(ColumnFamilyStore cfs, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs) throws java.lang.InterruptedException, java.util.concurrent.ExecutionException
java.lang.InterruptedException
java.util.concurrent.ExecutionException
public CompactionManager.AllSSTableOpStatus performVerify(ColumnFamilyStore cfs, Verifier.Options options) throws java.lang.InterruptedException, java.util.concurrent.ExecutionException
java.lang.InterruptedException
java.util.concurrent.ExecutionException
public CompactionManager.AllSSTableOpStatus performSSTableRewrite(ColumnFamilyStore cfs, boolean excludeCurrentVersion, int jobs) throws java.lang.InterruptedException, java.util.concurrent.ExecutionException
java.lang.InterruptedException
java.util.concurrent.ExecutionException
public CompactionManager.AllSSTableOpStatus performCleanup(ColumnFamilyStore cfStore, int jobs) throws java.lang.InterruptedException, java.util.concurrent.ExecutionException
java.lang.InterruptedException
java.util.concurrent.ExecutionException
public CompactionManager.AllSSTableOpStatus performGarbageCollection(ColumnFamilyStore cfStore, CompactionParams.TombstoneOption tombstoneOption, int jobs) throws java.lang.InterruptedException, java.util.concurrent.ExecutionException
java.lang.InterruptedException
java.util.concurrent.ExecutionException
public CompactionManager.AllSSTableOpStatus relocateSSTables(ColumnFamilyStore cfs, int jobs) throws java.util.concurrent.ExecutionException, java.lang.InterruptedException
java.util.concurrent.ExecutionException
java.lang.InterruptedException
public com.google.common.util.concurrent.ListenableFuture<?> submitPendingAntiCompaction(ColumnFamilyStore cfs, RangesAtEndpoint tokenRanges, Refs<SSTableReader> sstables, LifecycleTransaction txn, java.util.UUID sessionId, java.util.function.BooleanSupplier isCancelled)
public void performAnticompaction(ColumnFamilyStore cfs, RangesAtEndpoint replicas, Refs<SSTableReader> validatedForRepair, LifecycleTransaction txn, java.util.UUID sessionID, java.util.function.BooleanSupplier isCancelled) throws java.io.IOException
cfs
- replicas
- token ranges to be repairedvalidatedForRepair
- SSTables containing the repaired ranges. Should be referenced before passing them.sessionID
- the repair session we're anti-compacting forisCancelled
- function that indicates if active anti-compaction should be canceledjava.lang.InterruptedException
java.io.IOException
public void performMaximal(ColumnFamilyStore cfStore, boolean splitOutput)
public java.util.List<java.util.concurrent.Future<?>> submitMaximal(ColumnFamilyStore cfStore, int gcBefore, boolean splitOutput)
public void forceCompactionForTokenRange(ColumnFamilyStore cfStore, java.util.Collection<Range<Token>> ranges)
public void forceUserDefinedCompaction(java.lang.String dataFiles)
CompactionManagerMBean
forceUserDefinedCompaction
in interface CompactionManagerMBean
dataFiles
- a comma separated list of sstable file to compact.
must contain keyspace and columnfamily name in path(for 2.1+) or file name itself.public void forceUserDefinedCleanup(java.lang.String dataFiles)
CompactionManagerMBean
forceUserDefinedCleanup
in interface CompactionManagerMBean
dataFiles
- a comma separated list of sstable file to cleanup.
must contain keyspace and columnfamily name in path(for 2.1+) or file name itself.public java.util.concurrent.Future<?> submitUserDefined(ColumnFamilyStore cfs, java.util.Collection<Descriptor> dataFiles, int gcBefore)
public java.util.concurrent.Future<?> submitValidation(java.util.concurrent.Callable<java.lang.Object> validation)
public void disableAutoCompaction()
public static boolean needsCleanup(SSTableReader sstable, java.util.Collection<Range<Token>> ownedRanges)
public static SSTableWriter createWriter(ColumnFamilyStore cfs, java.io.File compactionFileLocation, long expectedBloomFilterSize, long repairedAt, java.util.UUID pendingRepair, boolean isTransient, SSTableReader sstable, LifecycleTransaction txn)
public static SSTableWriter createWriterForAntiCompaction(ColumnFamilyStore cfs, java.io.File compactionFileLocation, int expectedBloomFilterSize, long repairedAt, java.util.UUID pendingRepair, boolean isTransient, java.util.Collection<SSTableReader> sstables, ILifecycleTransaction txn)
public static CompactionIterator getAntiCompactionIterator(java.util.List<ISSTableScanner> scanners, CompactionController controller, int nowInSec, java.util.UUID timeUUID, ActiveCompactionsTracker activeCompactions, java.util.function.BooleanSupplier isCancelled)
public com.google.common.util.concurrent.ListenableFuture<?> submitIndexBuild(SecondaryIndexBuilder builder)
public java.util.concurrent.Future<?> submitCacheWrite(AutoSavingCache.Writer writer)
public java.util.List<SSTableReader> runIndexSummaryRedistribution(IndexSummaryRedistribution redistribution) throws java.io.IOException
java.io.IOException
public static int getDefaultGcBefore(ColumnFamilyStore cfs, int nowInSec)
public com.google.common.util.concurrent.ListenableFuture<java.lang.Long> submitViewBuilder(ViewBuilderTask task)
public int getActiveCompactions()
public void incrementAborted()
public void incrementCompactionsReduced()
public void incrementSstablesDropppedFromCompactions(long num)
public java.util.List<java.util.Map<java.lang.String,java.lang.String>> getCompactions()
CompactionManagerMBean
getCompactions
in interface CompactionManagerMBean
public java.util.List<java.lang.String> getCompactionSummary()
CompactionManagerMBean
getCompactionSummary
in interface CompactionManagerMBean
public javax.management.openmbean.TabularData getCompactionHistory()
CompactionManagerMBean
getCompactionHistory
in interface CompactionManagerMBean
public long getTotalBytesCompacted()
public long getTotalCompactionsCompleted()
public int getPendingTasks()
public long getCompletedTasks()
public void stopCompaction(java.lang.String type)
CompactionManagerMBean
type
.stopCompaction
in interface CompactionManagerMBean
type
- the type of compaction to stop. Can be one of:
- COMPACTION
- VALIDATION
- CLEANUP
- SCRUB
- INDEX_BUILDpublic void stopCompactionById(java.lang.String compactionId)
CompactionManagerMBean
stopCompactionById
in interface CompactionManagerMBean
compactionId
- Compaction ID of compaction to stop. Such IDs can be found in
the transaction log files whose name starts with compaction_,
located in the table transactions folder.public void setConcurrentCompactors(int value)
public void setConcurrentValidations()
public void setConcurrentViewBuilders(int value)
public int getCoreCompactorThreads()
CompactionManagerMBean
getCoreCompactorThreads
in interface CompactionManagerMBean
public void setCoreCompactorThreads(int number)
CompactionManagerMBean
setCoreCompactorThreads
in interface CompactionManagerMBean
number
- New maximum of compaction threadspublic int getMaximumCompactorThreads()
CompactionManagerMBean
getMaximumCompactorThreads
in interface CompactionManagerMBean
public void setMaximumCompactorThreads(int number)
CompactionManagerMBean
setMaximumCompactorThreads
in interface CompactionManagerMBean
number
- New maximum of compaction threadspublic int getCoreValidationThreads()
CompactionManagerMBean
getCoreValidationThreads
in interface CompactionManagerMBean
public void setCoreValidationThreads(int number)
CompactionManagerMBean
setCoreValidationThreads
in interface CompactionManagerMBean
number
- New maximum of compaction threadspublic int getMaximumValidatorThreads()
CompactionManagerMBean
getMaximumValidatorThreads
in interface CompactionManagerMBean
public void setMaximumValidatorThreads(int number)
CompactionManagerMBean
setMaximumValidatorThreads
in interface CompactionManagerMBean
number
- New maximum of validator threadspublic boolean getDisableSTCSInL0()
CompactionManagerMBean
getDisableSTCSInL0
in interface CompactionManagerMBean
public void setDisableSTCSInL0(boolean disabled)
setDisableSTCSInL0
in interface CompactionManagerMBean
public int getCoreViewBuildThreads()
CompactionManagerMBean
getCoreViewBuildThreads
in interface CompactionManagerMBean
public void setCoreViewBuildThreads(int number)
CompactionManagerMBean
setCoreViewBuildThreads
in interface CompactionManagerMBean
number
- New maximum of view build threadspublic int getMaximumViewBuildThreads()
CompactionManagerMBean
getMaximumViewBuildThreads
in interface CompactionManagerMBean
public void setMaximumViewBuildThreads(int number)
CompactionManagerMBean
setMaximumViewBuildThreads
in interface CompactionManagerMBean
number
- New maximum of view build threadspublic boolean getAutomaticSSTableUpgradeEnabled()
CompactionManagerMBean
getAutomaticSSTableUpgradeEnabled
in interface CompactionManagerMBean
public void setAutomaticSSTableUpgradeEnabled(boolean enabled)
CompactionManagerMBean
setAutomaticSSTableUpgradeEnabled
in interface CompactionManagerMBean
public int getMaxConcurrentAutoUpgradeTasks()
CompactionManagerMBean
getMaxConcurrentAutoUpgradeTasks
in interface CompactionManagerMBean
public void setMaxConcurrentAutoUpgradeTasks(int value)
CompactionManagerMBean
setMaxConcurrentAutoUpgradeTasks
in interface CompactionManagerMBean
public void interruptCompactionFor(java.lang.Iterable<TableMetadata> columnFamilies, java.util.function.Predicate<SSTableReader> sstablePredicate, boolean interruptValidation)
columnFamilies
- The ColumnFamilies to try to stop compaction upon.sstablePredicate
- the sstable predicate to match oninterruptValidation
- true if validation operations for repair should also be interruptedpublic void interruptCompactionForCFs(java.lang.Iterable<ColumnFamilyStore> cfss, java.util.function.Predicate<SSTableReader> sstablePredicate, boolean interruptValidation)
public void waitForCessation(java.lang.Iterable<ColumnFamilyStore> cfss, java.util.function.Predicate<SSTableReader> sstablePredicate)
public java.util.List<CompactionInfo> getSSTableTasks()
public boolean isGlobalCompactionPaused()
public CompactionManager.CompactionPauser pauseGlobalCompaction()
Copyright © 2009-2021 The Apache Software Foundation