{
return result;
}
mergedFileName = getTempFileName();
SSTable ssTableRange = null;
String lastkey = null;
List<FileStruct> lfs = new ArrayList<FileStruct>();
DataOutputBuffer bufOut = new DataOutputBuffer();
int expectedBloomFilterSize = SSTable.getApproximateKeyCount(files);
expectedBloomFilterSize = (expectedBloomFilterSize > 0) ? expectedBloomFilterSize : SSTable.indexInterval();
logger_.debug("Expected bloom filter size : " + expectedBloomFilterSize);
/* Create the bloom filter for the compacted file. */
BloomFilter compactedRangeBloomFilter = new BloomFilter(expectedBloomFilterSize, 15);
List<ColumnFamily> columnFamilies = new ArrayList<ColumnFamily>();
while (pq.size() > 0 || lfs.size() > 0)
{
FileStruct fs = null;
if (pq.size() > 0)
{
fs = pq.poll();
}
if (fs != null
&& (lastkey == null || lastkey.equals(fs.getKey())))
{
// The keys are the same so we need to add this to the
// ldfs list
lastkey = fs.getKey();
lfs.add(fs);
}
else
{
Collections.sort(lfs, new FileStructComparator());
ColumnFamily columnFamily;
bufOut.reset();
if (lfs.size() > 1)
{
for (FileStruct filestruct : lfs)
{
try
{
/* read the length although we don't need it */
filestruct.getBufIn().readInt();
// Skip the Index
IndexHelper.skipBloomFilterAndIndex(filestruct.getBufIn());
// We want to add only 2 and resolve them right there in order to save on memory footprint
if (columnFamilies.size() > 1)
{
// Now merge the 2 column families
merge(columnFamilies);
}
// deserialize into column families
columnFamilies.add(ColumnFamily.serializer().deserialize(filestruct.getBufIn()));
}
catch (Exception ex)
{
logger_.warn(LogUtil.throwableToString(ex));
}
}
// Now after merging all crap append to the sstable
columnFamily = resolveAndRemoveDeleted(columnFamilies);
columnFamilies.clear();
if (columnFamily != null)
{
/* serialize the cf with column indexes */
ColumnFamily.serializerWithIndexes().serialize(columnFamily, bufOut);
}
}
else
{
FileStruct filestruct = lfs.get(0);
/* read the length although we don't need it */
int size = filestruct.getBufIn().readInt();
bufOut.write(filestruct.getBufIn(), size);
}
if (Range.isTokenInRanges(StorageService.getPartitioner().getInitialToken(lastkey), ranges))
{
if (ssTableRange == null)
{
if (target != null)
{
rangeFileLocation = rangeFileLocation + System.getProperty("file.separator") + "bootstrap";
}
FileUtils.createDirectory(rangeFileLocation);
ssTableRange = new SSTable(rangeFileLocation, mergedFileName, StorageService.getPartitioner());
}
try
{
ssTableRange.append(lastkey, bufOut);
compactedRangeBloomFilter.add(lastkey);
}
catch (Exception ex)
{
logger_.warn(LogUtil.throwableToString(ex));
}
}
totalkeysWritten++;
for (FileStruct filestruct : lfs)
{
try
{
filestruct.advance();
if (filestruct.isExhausted())
{
continue;
}
/* keep on looping until we find a key in the range */
while (!Range.isTokenInRanges(StorageService.getPartitioner().getInitialToken(filestruct.getKey()), ranges))
{
filestruct.advance();
if (filestruct.isExhausted())
{
break;
}
}
if (!filestruct.isExhausted())
{
pq.add(filestruct);
}
totalkeysRead++;
}
catch (Exception ex)
{
// Ignore the exception as it might be a corrupted file
// in any case we have read as far as possible from it
// and it will be deleted after compaction.
logger_.warn("corrupt sstable?", ex);
filestruct.close();
}
}
lfs.clear();
lastkey = null;
if (fs != null)
{
// Add back the fs since we processed the rest of
// filestructs
pq.add(fs);
}
}
}
if (ssTableRange != null)
{
ssTableRange.closeRename(compactedRangeBloomFilter);
if (fileList != null)
{
fileList.add(ssTableRange.getDataFileLocation());
}
if (compactedBloomFilters != null)
{
compactedBloomFilters.add(compactedRangeBloomFilter);
}