|
@@ -28,6 +28,7 @@ import java.net.InetSocketAddress;
|
|
|
import java.net.URISyntaxException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Iterator;
|
|
@@ -567,26 +568,14 @@ public class ResourceLocalizationService extends CompositeService
|
|
|
final Configuration conf;
|
|
|
final ExecutorService threadPool;
|
|
|
final CompletionService<Path> queue;
|
|
|
+ // Its shared between public localizer and dispatcher thread.
|
|
|
final Map<Future<Path>,LocalizerResourceRequestEvent> pending;
|
|
|
// TODO hack to work around broken signaling
|
|
|
final Map<LocalResourceRequest,List<LocalizerResourceRequestEvent>> attempts;
|
|
|
|
|
|
PublicLocalizer(Configuration conf) {
|
|
|
- this(conf, getLocalFileContext(conf),
|
|
|
- createLocalizerExecutor(conf),
|
|
|
- new HashMap<Future<Path>,LocalizerResourceRequestEvent>(),
|
|
|
- new HashMap<LocalResourceRequest,List<LocalizerResourceRequestEvent>>());
|
|
|
- }
|
|
|
-
|
|
|
- PublicLocalizer(Configuration conf, FileContext lfs,
|
|
|
- ExecutorService threadPool,
|
|
|
- Map<Future<Path>,LocalizerResourceRequestEvent> pending,
|
|
|
- Map<LocalResourceRequest,List<LocalizerResourceRequestEvent>> attempts) {
|
|
|
- super("Public Localizer");
|
|
|
- this.lfs = lfs;
|
|
|
+ this.lfs = getLocalFileContext(conf);
|
|
|
this.conf = conf;
|
|
|
- this.pending = pending;
|
|
|
- this.attempts = attempts;
|
|
|
// List<String> localDirs = dirsHandler.getLocalDirs();
|
|
|
// String[] publicFilecache = new String[localDirs.size()];
|
|
|
// for (int i = 0, n = localDirs.size(); i < n; ++i) {
|
|
@@ -604,7 +593,11 @@ public class ResourceLocalizationService extends CompositeService
|
|
|
// new Path(localDir, ContainerLocalizer.FILECACHE).toString();
|
|
|
// }
|
|
|
|
|
|
- this.threadPool = threadPool;
|
|
|
+ this.pending =
|
|
|
+ new ConcurrentHashMap<Future<Path>, LocalizerResourceRequestEvent>();
|
|
|
+ this.attempts =
|
|
|
+ new HashMap<LocalResourceRequest,List<LocalizerResourceRequestEvent>>();
|
|
|
+ this.threadPool = createLocalizerExecutor(conf);
|
|
|
this.queue = new ExecutorCompletionService<Path>(threadPool);
|
|
|
}
|
|
|
|
|
@@ -708,6 +701,7 @@ public class ResourceLocalizationService extends CompositeService
|
|
|
final LocalizerContext context;
|
|
|
final String localizerId;
|
|
|
final Map<LocalResourceRequest,LocalizerResourceRequestEvent> scheduled;
|
|
|
+ // Its a shared list between Private Localizer and dispatcher thread.
|
|
|
final List<LocalizerResourceRequestEvent> pending;
|
|
|
|
|
|
// TODO: threadsafe, use outer?
|
|
@@ -718,13 +712,14 @@ public class ResourceLocalizationService extends CompositeService
|
|
|
super("LocalizerRunner for " + localizerId);
|
|
|
this.context = context;
|
|
|
this.localizerId = localizerId;
|
|
|
- this.pending = new ArrayList<LocalizerResourceRequestEvent>();
|
|
|
+ this.pending =
|
|
|
+ Collections
|
|
|
+ .synchronizedList(new ArrayList<LocalizerResourceRequestEvent>());
|
|
|
this.scheduled =
|
|
|
new HashMap<LocalResourceRequest, LocalizerResourceRequestEvent>();
|
|
|
}
|
|
|
|
|
|
public void addResource(LocalizerResourceRequestEvent request) {
|
|
|
- // TDOO: Synchronization
|
|
|
pending.add(request);
|
|
|
}
|
|
|
|
|
@@ -734,30 +729,31 @@ public class ResourceLocalizationService extends CompositeService
|
|
|
* @return
|
|
|
*/
|
|
|
private LocalResource findNextResource() {
|
|
|
- // TODO: Synchronization
|
|
|
- for (Iterator<LocalizerResourceRequestEvent> i = pending.iterator();
|
|
|
- i.hasNext();) {
|
|
|
- LocalizerResourceRequestEvent evt = i.next();
|
|
|
- LocalizedResource nRsrc = evt.getResource();
|
|
|
- if (ResourceState.LOCALIZED.equals(nRsrc.getState())) {
|
|
|
- i.remove();
|
|
|
- continue;
|
|
|
- }
|
|
|
- if (nRsrc.tryAcquire()) {
|
|
|
- LocalResourceRequest nextRsrc = nRsrc.getRequest();
|
|
|
- LocalResource next =
|
|
|
- recordFactory.newRecordInstance(LocalResource.class);
|
|
|
- next.setResource(
|
|
|
- ConverterUtils.getYarnUrlFromPath(nextRsrc.getPath()));
|
|
|
- next.setTimestamp(nextRsrc.getTimestamp());
|
|
|
- next.setType(nextRsrc.getType());
|
|
|
- next.setVisibility(evt.getVisibility());
|
|
|
- next.setPattern(evt.getPattern());
|
|
|
- scheduled.put(nextRsrc, evt);
|
|
|
- return next;
|
|
|
+ synchronized (pending) {
|
|
|
+ for (Iterator<LocalizerResourceRequestEvent> i = pending.iterator();
|
|
|
+ i.hasNext();) {
|
|
|
+ LocalizerResourceRequestEvent evt = i.next();
|
|
|
+ LocalizedResource nRsrc = evt.getResource();
|
|
|
+ if (ResourceState.LOCALIZED.equals(nRsrc.getState())) {
|
|
|
+ i.remove();
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ if (nRsrc.tryAcquire()) {
|
|
|
+ LocalResourceRequest nextRsrc = nRsrc.getRequest();
|
|
|
+ LocalResource next =
|
|
|
+ recordFactory.newRecordInstance(LocalResource.class);
|
|
|
+ next.setResource(ConverterUtils.getYarnUrlFromPath(nextRsrc
|
|
|
+ .getPath()));
|
|
|
+ next.setTimestamp(nextRsrc.getTimestamp());
|
|
|
+ next.setType(nextRsrc.getType());
|
|
|
+ next.setVisibility(evt.getVisibility());
|
|
|
+ next.setPattern(evt.getPattern());
|
|
|
+ scheduled.put(nextRsrc, evt);
|
|
|
+ return next;
|
|
|
+ }
|
|
|
}
|
|
|
+ return null;
|
|
|
}
|
|
|
- return null;
|
|
|
}
|
|
|
|
|
|
// TODO this sucks. Fix it later
|