public float getProgress() { QueryState state = getStateMachine().getCurrentState(); if (state == QueryState.QUERY_SUCCEEDED) { return 1.0f; } else { int idx = 0; List<SubQuery> tempSubQueries = new ArrayList<SubQuery>(); synchronized (subqueries) { tempSubQueries.addAll(subqueries.values()); } float[] subProgresses = new float[tempSubQueries.size()]; boolean finished = true; for (SubQuery subquery : tempSubQueries) { if (subquery.getState() != SubQueryState.NEW) { subProgresses[idx] = subquery.getProgress(); if (finished && subquery.getState() != SubQueryState.SUCCEEDED) { finished = false; } } else { subProgresses[idx] = 0.0f; } idx++; } if (finished) { return 1.0f; } float totalProgress = 0; float proportion = 1.0f / (float) (getExecutionBlockCursor().size() - 1); // minus one is due to for (int i = 0; i < subProgresses.length; i++) { totalProgress += subProgresses[i] * proportion; } return totalProgress; } }
private TaskAttemptId allocateRackTask(String host) { List<HostVolumeMapping> remainingTasks = Lists.newArrayList(leafTaskHostMapping.values()); String rack = RackResolver.resolve(host).getNetworkLocation(); TaskAttemptId attemptId = null; if (remainingTasks.size() > 0) { synchronized (scheduledRequests) { // find largest remaining task of other host in rack Collections.sort( remainingTasks, new Comparator<HostVolumeMapping>() { @Override public int compare(HostVolumeMapping v1, HostVolumeMapping v2) { // descending remaining tasks if (v2.remainTasksNum.get() > v1.remainTasksNum.get()) { return 1; } else if (v2.remainTasksNum.get() == v1.remainTasksNum.get()) { return 0; } else { return -1; } } }); } for (HostVolumeMapping tasks : remainingTasks) { for (int i = 0; i < tasks.getRemainingLocalTaskSize(); i++) { TaskAttemptId tId = tasks.getTaskAttemptIdByRack(rack); if (tId == null) break; if (leafTasks.contains(tId)) { leafTasks.remove(tId); attemptId = tId; break; } } if (attemptId != null) break; } } // find task in rack if (attemptId == null) { HashSet<TaskAttemptId> list = leafTasksRackMapping.get(rack); if (list != null) { synchronized (list) { Iterator<TaskAttemptId> iterator = list.iterator(); while (iterator.hasNext()) { TaskAttemptId tId = iterator.next(); iterator.remove(); if (leafTasks.contains(tId)) { leafTasks.remove(tId); attemptId = tId; break; } } } } } return attemptId; }