package org.apache.cassandra.streaming;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.io.sstable.SSTableMultiWriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.streaming.messages.OutgoingFileMessage;
import org.apache.cassandra.utils.FBUtilities;
public interface StreamHook
{
public static final StreamHook instance = createHook();
public OutgoingFileMessage reportOutgoingFile(StreamSession session, SSTableReader sstable, OutgoingFileMessage message);
public void reportStreamFuture(StreamSession session, StreamResultFuture future);
public void reportIncomingFile(ColumnFamilyStore cfs, SSTableMultiWriter writer, StreamSession session, int sequenceNumber);
static StreamHook createHook()
{
String className = System.getProperty("cassandra.stream_hook");
if (className != null)
{
return FBUtilities.construct(className, StreamHook.class.getSimpleName());
}
else
{
return new StreamHook()
{
public OutgoingFileMessage reportOutgoingFile(StreamSession session, SSTableReader sstable, OutgoingFileMessage message)
{
return message;
}
public void reportStreamFuture(StreamSession session, StreamResultFuture future) {}
public void reportIncomingFile(ColumnFamilyStore cfs, SSTableMultiWriter writer, StreamSession session, int sequenceNumber) {}
};
}
}
}