mirror of https://github.com/apache/druid.git
make find sinks to persist run in one callable together with the actual persist work
This commit is contained in:
parent
c9a6de7fc5
commit
a24a2d80ae
|
@ -90,7 +90,6 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -482,16 +481,24 @@ public class AppenderatorImpl implements Appenderator
|
|||
{
|
||||
throwPersistErrorIfExists();
|
||||
|
||||
log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource());
|
||||
|
||||
final String threadName = StringUtils.format("%s-incremental-persist", schema.getDataSource());
|
||||
final Object commitMetadata = committer == null ? null : committer.getMetadata();
|
||||
final Stopwatch runExecStopwatch = Stopwatch.createStarted();
|
||||
final Stopwatch persistStopwatch = Stopwatch.createStarted();
|
||||
final ListenableFuture<Object> future = persistExecutor.submit(
|
||||
new ThreadRenamingCallable<Object>(threadName)
|
||||
{
|
||||
@Override
|
||||
public Object doCall() throws IOException
|
||||
{
|
||||
final Map<String, Integer> currentHydrants = new HashMap<>();
|
||||
final List<Pair<FireHydrant, SegmentIdentifier>> indexesToPersist = new ArrayList<>();
|
||||
int numPersistedRows = 0;
|
||||
long bytesPersisted = 0L;
|
||||
Iterator<Map.Entry<SegmentIdentifier, Sink>> iterator = sinks.entrySet().iterator();
|
||||
|
||||
while (iterator.hasNext()) {
|
||||
final Map.Entry<SegmentIdentifier, Sink> entry = iterator.next();
|
||||
final SegmentIdentifier identifier = entry.getKey();
|
||||
final Sink sink = entry.getValue();
|
||||
for (SegmentIdentifier identifier : sinks.keySet()) {
|
||||
final Sink sink = sinks.get(identifier);
|
||||
if (sink == null) {
|
||||
throw new ISE("No sink for identifier: %s", identifier);
|
||||
}
|
||||
|
@ -513,19 +520,10 @@ public class AppenderatorImpl implements Appenderator
|
|||
indexesToPersist.add(Pair.of(sink.swap(), identifier));
|
||||
}
|
||||
}
|
||||
// NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes.
|
||||
rowsCurrentlyInMemory.addAndGet(-numPersistedRows);
|
||||
bytesCurrentlyInMemory.addAndGet(-bytesPersisted);
|
||||
|
||||
log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource());
|
||||
|
||||
final String threadName = StringUtils.format("%s-incremental-persist", schema.getDataSource());
|
||||
final Object commitMetadata = committer == null ? null : committer.getMetadata();
|
||||
final Stopwatch runExecStopwatch = Stopwatch.createStarted();
|
||||
final Stopwatch persistStopwatch = Stopwatch.createStarted();
|
||||
final ListenableFuture<Object> future = persistExecutor.submit(
|
||||
new ThreadRenamingCallable<Object>(threadName)
|
||||
{
|
||||
@Override
|
||||
public Object doCall() throws IOException
|
||||
{
|
||||
try {
|
||||
for (Pair<FireHydrant, SegmentIdentifier> pair : indexesToPersist) {
|
||||
metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs));
|
||||
|
@ -587,9 +585,6 @@ public class AppenderatorImpl implements Appenderator
|
|||
runExecStopwatch.stop();
|
||||
resetNextFlush();
|
||||
|
||||
// NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes.
|
||||
rowsCurrentlyInMemory.addAndGet(-numPersistedRows);
|
||||
bytesCurrentlyInMemory.addAndGet(-bytesPersisted);
|
||||
return future;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue