}
@Override
public void executeWithCallback(final RowCallback<K, C> callback) throws ConnectionException {
final ThriftKeyspaceImpl keyspace = query.keyspace;
final Partitioner partitioner = keyspace.getPartitioner();
final AtomicReference<ConnectionException> error = new AtomicReference<ConnectionException>();
final boolean bIgnoreTombstones = shouldIgnoreEmptyRows();
List<Pair<String, String>> ranges;
if (this.getConcurrencyLevel() != null) {
ranges = Lists.newArrayList();
int nThreads = this.getConcurrencyLevel();
List<TokenRange> tokens = partitioner.splitTokenRange(
startToken == null ? partitioner.getMinToken() : startToken,
endToken == null ? partitioner.getMaxToken() : endToken,
nThreads);
for (TokenRange range : tokens) {
try {
String currentToken = checkpointManager.getCheckpoint(range.getStartToken());
if (currentToken == null) {
currentToken = range.getStartToken();
}
else if (currentToken.equals(range.getEndToken())) {
continue;
}
ranges.add(Pair.create(currentToken, range.getEndToken()));
} catch (Exception e) {
throw ThriftConverter.ToConnectionPoolException(e);
}
}
}
else {
ranges = Lists.transform(keyspace.describeRing(true), new Function<TokenRange, Pair<String, String>> () {
@Override
public Pair<String, String> apply(TokenRange input) {
return Pair.create(input.getStartToken(), input.getEndToken());
}
});
}
final CountDownLatch doneSignal = new CountDownLatch(ranges.size());
for (final Pair<String, String> tokenPair : ranges) {
// Prepare the range of tokens for this token range
final KeyRange range = new KeyRange()
.setCount(getBlockSize())
.setStart_token(tokenPair.left)
.setEnd_token(tokenPair.right);
query.executor.submit(new Callable<Void>() {
private boolean firstBlock = true;
@Override
public Void call() throws Exception {
if (error.get() == null && internalRun()) {
query.executor.submit(this);
}
else {
doneSignal.countDown();
}
return null;
}
private boolean internalRun() throws Exception {
try {
// Get the next block
List<KeySlice> ks = keyspace.connectionPool.executeWithFailover(
new AbstractKeyspaceOperationImpl<List<KeySlice>>(keyspace.tracerFactory
.newTracer(CassandraOperationType.GET_ROWS_RANGE,
columnFamily), query.pinnedHost, keyspace
.getKeyspaceName()) {
@Override
public List<KeySlice> internalExecute(Client client, ConnectionContext context)
throws Exception {
return client.get_range_slices(new ColumnParent()
.setColumn_family(columnFamily.getName()),
predicate, range, ThriftConverter
.ToThriftConsistencyLevel(query.consistencyLevel));
}
@Override
public ByteBuffer getRowKey() {
if (range.getStart_key() != null)
return ByteBuffer.wrap(range.getStart_key());
return null;
}
}, query.retry.duplicate()).getResult();
// Notify the callback
if (!ks.isEmpty()) {
KeySlice lastRow = Iterables.getLast(ks);
boolean bContinue = (ks.size() == getBlockSize());
if (getRepeatLastToken()) {
if (firstBlock) {
firstBlock = false;
}
else {
ks.remove(0);
}
}
if (bIgnoreTombstones) {
Iterator<KeySlice> iter = ks.iterator();
while (iter.hasNext()) {
if (iter.next().getColumnsSize() == 0)
iter.remove();
}
}
Rows<K, C> rows = new ThriftRowsSliceImpl<K, C>(ks, columnFamily
.getKeySerializer(), columnFamily.getColumnSerializer());
try {
callback.success(rows);
}
catch (Throwable t) {
ConnectionException ce = ThriftConverter.ToConnectionPoolException(t);
error.set(ce);
return false;
}
if (bContinue) {
// Determine the start token for the next page
String token = partitioner.getTokenForKey(lastRow.bufferForKey()).toString();
checkpointManager.trackCheckpoint(tokenPair.left, token);
if (getRepeatLastToken()) {
range.setStart_token(partitioner.getTokenMinusOne(token));
}
else {
range.setStart_token(token);
}
}