Skip to content

Commit

Permalink
merge from 1.0
Browse files Browse the repository at this point in the history
  • Loading branch information
xedin committed Mar 22, 2012
2 parents 3136c20 + fbdf7b0 commit b12c34f
Show file tree
Hide file tree
Showing 3 changed files with 44 additions and 21 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
Expand Up @@ -38,6 +38,7 @@ Merged from 1.0:
* don't change manifest level for cleanup, scrub, and upgradesstables
operations under LeveledCompactionStrategy (CASSANDRA-3989)
* fix race leading to super columns assertion failure (CASSANDRA-3957)
* ensure that directory is selected for compaction (CASSANDRA-3985)


1.1-beta1
Expand Down
32 changes: 26 additions & 6 deletions src/java/org/apache/cassandra/db/Directories.java
Expand Up @@ -112,7 +112,12 @@ private Directories(String tablename, String cfname, String directoryName)

public File getDirectoryForNewSSTables(long estimatedSize)
{
File path = getLocationWithMaximumAvailableSpace(estimatedSize);
return getDirectoryForNewSSTables(estimatedSize, true);
}

public File getDirectoryForNewSSTables(long estimatedSize, boolean ensureFreeSpace)
{
File path = getLocationWithMaximumAvailableSpace(estimatedSize, ensureFreeSpace);
// Requesting GC has a chance to free space only if we're using mmap and a non SUN jvm
if (path == null
&& (DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap || DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
Expand All @@ -130,7 +135,7 @@ public File getDirectoryForNewSSTables(long estimatedSize)
{
throw new AssertionError(e);
}
path = getLocationWithMaximumAvailableSpace(estimatedSize);
path = getLocationWithMaximumAvailableSpace(estimatedSize, ensureFreeSpace);
}
return path;
}
Expand All @@ -141,7 +146,7 @@ public File getDirectoryForNewSSTables(long estimatedSize)
* compacted file is greater than the max disk space available return null, we cannot
* do compaction in this case.
*/
public File getLocationWithMaximumAvailableSpace(long estimatedSize)
public File getLocationWithMaximumAvailableSpace(long estimatedSize, boolean ensureFreeSpace)
{
long maxFreeDisk = 0;
File maxLocation = null;
Expand All @@ -154,11 +159,26 @@ public File getLocationWithMaximumAvailableSpace(long estimatedSize)
maxLocation = dir;
}
}
logger.debug(String.format("expected data files size is %d; largest free partition (%s) has %d bytes free", estimatedSize, maxLocation, maxFreeDisk));
logger.debug(String.format("expected data files size is %d; largest free partition (%s) has %d bytes free",
estimatedSize,
maxLocation,
maxFreeDisk));

// Load factor of 0.9 we do not want to use the entire disk that is too risky.
maxFreeDisk = (long)(0.9 * maxFreeDisk);
return estimatedSize < maxFreeDisk ? maxLocation : null;
maxFreeDisk = (long) (0.9 * maxFreeDisk);

if (!ensureFreeSpace || estimatedSize < maxFreeDisk)
{
if (estimatedSize >= maxFreeDisk)
logger.warn(String.format("Data file location %s only has %d free, estimated size is %d",
maxLocation,
maxFreeDisk,
estimatedSize));

return maxLocation;
}

return null;
}

public static File getSnapshotDirectory(Descriptor desc, String snapshotName)
Expand Down
32 changes: 17 additions & 15 deletions src/java/org/apache/cassandra/db/compaction/CompactionTask.java
Expand Up @@ -75,33 +75,30 @@ public int execute(CompactionExecutorStatsCollector collector) throws IOExceptio
if (!isCompactionInteresting(toCompact))
return 0;

// If use defined, we don't want to "trust" our space estimation. If
// there isn't enough room, it's the user problem
long expectedSize = isUserDefined ? 0 : cfs.getExpectedCompactedFileSize(toCompact);
File compactionFileLocation = cfs.directories.getDirectoryForNewSSTables(expectedSize);
if (partialCompactionsAcceptable())
File compactionFileLocation = cfs.directories.getDirectoryForNewSSTables(cfs.getExpectedCompactedFileSize(toCompact),
ensureFreeSpace());

if (compactionFileLocation == null && partialCompactionsAcceptable())
{
// If the compaction file path is null that means we have no space left for this compaction.
// Try again w/o the largest one.
if (compactionFileLocation == null)
while (compactionFileLocation == null && toCompact.size() > 1)
{
while (compactionFileLocation == null && toCompact.size() > 1)
{
logger.warn("insufficient space to compact all requested files " + StringUtils.join(toCompact, ", "));
// Note that we have removed files that are still marked as compacting. This suboptimal but ok since the caller will unmark all
// the sstables at the end.
toCompact.remove(cfs.getMaxSizeFile(toCompact));
compactionFileLocation = cfs.directories.getDirectoryForNewSSTables(cfs.getExpectedCompactedFileSize(toCompact));
}
logger.warn("insufficient space to compact all requested files " + StringUtils.join(toCompact, ", "));
// Note that we have removed files that are still marked as compacting.
// This suboptimal but ok since the caller will unmark all the sstables at the end.
toCompact.remove(cfs.getMaxSizeFile(toCompact));
compactionFileLocation = cfs.directories.getDirectoryForNewSSTables(cfs.getExpectedCompactedFileSize(toCompact),
ensureFreeSpace());
}

}

if (compactionFileLocation == null)
{
logger.warn("insufficient space to compact even the two smallest files, aborting");
return 0;
}
assert compactionFileLocation != null;

if (DatabaseDescriptor.isSnapshotBeforeCompaction())
cfs.snapshotWithoutFlush(System.currentTimeMillis() + "-" + "compact-" + cfs.columnFamily);
Expand Down Expand Up @@ -236,6 +233,11 @@ protected boolean partialCompactionsAcceptable()
return !isUserDefined;
}

protected boolean ensureFreeSpace()
{
return !isUserDefined;
}

//extensibility point for other strategies that may want to limit the upper bounds of the sstable segment size
protected boolean newSSTableSegmentThresholdReached(SSTableWriter writer, long position)
{
Expand Down

0 comments on commit b12c34f

Please sign in to comment.