public synchronized void flush() throws MutationsRejectedException { if (closed) throw new IllegalStateException("Closed"); Span span = Trace.start("flush"); try { checkForFailures(); if (flushing) { // some other thread is currently flushing, so wait while (flushing && !somethingFailed) waitRTE(); checkForFailures(); return; } flushing = true; startProcessing(); checkForFailures(); while (totalMemUsed > 0 && !somethingFailed) { waitRTE(); } flushing = false; this.notifyAll(); checkForFailures(); } finally { span.stop(); } }
void addMutations(MutationSet mutationsToSend) { Map<String, TabletServerMutations<Mutation>> binnedMutations = new HashMap<String, TabletServerMutations<Mutation>>(); Span span = Trace.start("binMutations"); try { long t1 = System.currentTimeMillis(); binMutations(mutationsToSend, binnedMutations); long t2 = System.currentTimeMillis(); updateBinningStats(mutationsToSend.size(), (t2 - t1), binnedMutations); } finally { span.stop(); } addMutations(binnedMutations); }
private synchronized void addMutations( Map<String, TabletServerMutations<Mutation>> binnedMutations) { int count = 0; // merge mutations into existing mutations for a tablet server for (Entry<String, TabletServerMutations<Mutation>> entry : binnedMutations.entrySet()) { String server = entry.getKey(); TabletServerMutations<Mutation> currentMutations = serversMutations.get(server); if (currentMutations == null) { serversMutations.put(server, entry.getValue()); } else { for (Entry<KeyExtent, List<Mutation>> entry2 : entry.getValue().getMutations().entrySet()) { for (Mutation m : entry2.getValue()) { currentMutations.addMutation(entry2.getKey(), m); } } } if (log.isTraceEnabled()) for (Entry<KeyExtent, List<Mutation>> entry2 : entry.getValue().getMutations().entrySet()) count += entry2.getValue().size(); } if (count > 0 && log.isTraceEnabled()) log.trace( String.format( "Started sending %,d mutations to %,d tablet servers", count, binnedMutations.keySet().size())); // randomize order of servers ArrayList<String> servers = new ArrayList<String>(binnedMutations.keySet()); Collections.shuffle(servers); for (String server : servers) if (!queued.contains(server)) { sendThreadPool.submit(Trace.wrap(new SendTask(server))); queued.add(server); } }
public synchronized void close() throws MutationsRejectedException { if (closed) return; Span span = Trace.start("close"); try { closed = true; startProcessing(); while (totalMemUsed > 0 && !somethingFailed) { waitRTE(); } logStats(); checkForFailures(); } finally { // make a best effort to release these resources writer.sendThreadPool.shutdownNow(); jtimer.cancel(); span.stop(); } }
public void send(TabletServerMutations<Mutation> tsm) throws AccumuloServerException, AccumuloSecurityException { MutationSet failures = null; String oldName = Thread.currentThread().getName(); Map<KeyExtent, List<Mutation>> mutationBatch = tsm.getMutations(); try { long count = 0; for (List<Mutation> list : mutationBatch.values()) { count += list.size(); } String msg = "sending " + String.format("%,d", count) + " mutations to " + String.format("%,d", mutationBatch.size()) + " tablets at " + location; Thread.currentThread().setName(msg); Span span = Trace.start("sendMutations"); try { TimeoutTracker timeoutTracker = timeoutTrackers.get(location); if (timeoutTracker == null) { timeoutTracker = new TimeoutTracker(location, timeout); timeoutTrackers.put(location, timeoutTracker); } long st1 = System.currentTimeMillis(); failures = sendMutationsToTabletServer(location, mutationBatch, timeoutTracker); long st2 = System.currentTimeMillis(); if (log.isTraceEnabled()) log.trace( "sent " + String.format("%,d", count) + " mutations to " + location + " in " + String.format( "%.2f secs (%,.2f mutations/sec) with %,d failures", (st2 - st1) / 1000.0, count / ((st2 - st1) / 1000.0), failures.size())); long successBytes = 0; for (Entry<KeyExtent, List<Mutation>> entry : mutationBatch.entrySet()) { for (Mutation mutation : entry.getValue()) { successBytes += mutation.estimatedMemoryUsed(); } } if (failures.size() > 0) { failedMutations.add(failures); successBytes -= failures.getMemoryUsed(); } updateSendStats(count, st2 - st1); decrementMemUsed(successBytes); } finally { span.stop(); } } catch (IOException e) { if (log.isTraceEnabled()) log.trace("failed to send mutations to " + location + " : " + e.getMessage()); HashSet<String> tables = new HashSet<String>(); for (KeyExtent ke : mutationBatch.keySet()) tables.add(ke.getTableId().toString()); for (String table : tables) TabletLocator.getLocator(instance, new Text(table)).invalidateCache(location); failedMutations.add(location, tsm); } finally { Thread.currentThread().setName(oldName); } }