else
client = ThriftUtil.getTServerClient(server, conf);
try {
OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Starting multi scan, tserver=" + server + " #tablets=" + requested.size() + " #ranges="
+ sumSizes(requested.values()) + " ssil=" + options.serverSideIteratorList + " ssio=" + options.serverSideIteratorOptions);
TabletType ttype = TabletType.type(requested.keySet());
boolean waitForWrites = !ThriftScanner.serversWaitedForWrites.get(ttype).contains(server);
Map<TKeyExtent,List<TRange>> thriftTabletRanges = Translator.translate(requested, Translator.KET, new Translator.ListTranslator<Range,TRange>(
Translator.RT));
InitialMultiScan imsr = client.startMultiScan(Tracer.traceInfo(), credentials, thriftTabletRanges, Translator.translate(columns, Translator.CT),
options.serverSideIteratorList, options.serverSideIteratorOptions, ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()), waitForWrites);
if (waitForWrites)
ThriftScanner.serversWaitedForWrites.get(ttype).add(server);
MultiScanResult scanResult = imsr.result;
opTimer.stop("Got 1st multi scan results, #results=" + scanResult.results.size() + (scanResult.more ? " scanID=" + imsr.scanID : "")
+ " in %DURATION%");
ArrayList<Entry<Key,Value>> entries = new ArrayList<Map.Entry<Key,Value>>(scanResult.results.size());
for (TKeyValue kv : scanResult.results) {
entries.add(new MyEntry(new Key(kv.key), new Value(kv.value)));
}
if (entries.size() > 0)
receiver.receive(entries);
if (entries.size() > 0 || scanResult.fullScans.size() > 0)
timeoutTracker.madeProgress();
trackScanning(failures, unscanned, scanResult);
while (scanResult.more) {
timeoutTracker.check();
opTimer.start("Continuing multi scan, scanid=" + imsr.scanID);
scanResult = client.continueMultiScan(Tracer.traceInfo(), imsr.scanID);
opTimer.stop("Got more multi scan results, #results=" + scanResult.results.size() + (scanResult.more ? " scanID=" + imsr.scanID : "")
+ " in %DURATION%");
entries = new ArrayList<Map.Entry<Key,Value>>(scanResult.results.size());
for (TKeyValue kv : scanResult.results) {
entries.add(new MyEntry(new Key(kv.key), new Value(kv.value)));