private SSTableReader streamIn(DataInput input, PendingFile localFile, PendingFile remoteFile) throws IOException
{
ColumnFamilyStore cfs = Table.open(localFile.desc.ksname).getColumnFamilyStore(localFile.desc.cfname);
DecoratedKey key;
SSTableWriter writer = new SSTableWriter(localFile.getFilename(), remoteFile.estimatedKeys);
CompactionController controller = null;
try
{
BytesReadTracker in = new BytesReadTracker(input);
for (Pair<Long, Long> section : localFile.sections)
{
long length = section.right - section.left;
long bytesRead = 0;
while (bytesRead < length)
{
in.reset(0);
key = SSTableReader.decodeKey(StorageService.getPartitioner(), localFile.desc, ByteBufferUtil.readWithShortLength(in));
long dataSize = SSTableReader.readRowSize(in, localFile.desc);
ColumnFamily cached = cfs.getRawCachedRow(key);
if (cached != null && remoteFile.type == OperationType.AES && dataSize <= DatabaseDescriptor.getInMemoryCompactionLimit())
{
// need to update row cache
if (controller == null)
controller = new CompactionController(cfs, Collections.<SSTableReader>emptyList(), Integer.MIN_VALUE, true);
SSTableIdentityIterator iter = new SSTableIdentityIterator(cfs.metadata, in, key, 0, dataSize, true);
PrecompactedRow row = new PrecompactedRow(controller, Collections.singletonList(iter));
// We don't expire anything so the row shouldn't be empty
assert !row.isEmpty();
writer.append(row);