@Override public Set<ChannelID> getInputChannelIDsOfGate(final GateID gateID) { InputGate<? extends IOReadableWritable> inputGate = null; final Iterator<InputGate<? extends IOReadableWritable>> gateIterator = this.inputGates.iterator(); while (gateIterator.hasNext()) { final InputGate<? extends IOReadableWritable> candidateGate = gateIterator.next(); if (candidateGate.getGateID().equals(gateID)) { inputGate = candidateGate; break; } } if (inputGate == null) { throw new IllegalArgumentException("Cannot find input gate with ID " + gateID); } final Set<ChannelID> inputChannelIDs = new HashSet<ChannelID>(); for (int i = 0; i < inputGate.getNumberOfInputChannels(); ++i) { inputChannelIDs.add(inputGate.getInputChannel(i).getID()); } return Collections.unmodifiableSet(inputChannelIDs); }
/** * Blocks until all input channels are closed. * * @throws IOException thrown if an error occurred while closing the input channels * @throws InterruptedException thrown if the thread waiting for the channels to be closed is * interrupted */ private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException { // Wait for disconnection of all output gates while (true) { // Make sure, we leave this method with an InterruptedException when the task has been // canceled if (this.executionObserver.isCanceled()) { throw new InterruptedException(); } boolean allClosed = true; for (int i = 0; i < getNumberOfInputGates(); i++) { final InputGate<? extends IOReadableWritable> eig = this.inputGates.get(i); if (!eig.isClosed()) { allClosed = false; } } if (allClosed) { break; } else { Thread.sleep(SLEEPINTERVAL); } } }
/** Closes all input gates which are not already closed. */ private void closeInputGates() throws IOException, InterruptedException { for (int i = 0; i < this.inputGates.size(); i++) { final InputGate<? extends IOReadableWritable> eig = this.inputGates.get(i); // Important: close must be called on each input gate exactly once eig.close(); } }
@Override public Set<ChannelID> getInputChannelIDs() { final Set<ChannelID> inputChannelIDs = new HashSet<ChannelID>(); final Iterator<InputGate<? extends IOReadableWritable>> gateIterator = this.inputGates.iterator(); while (gateIterator.hasNext()) { final InputGate<? extends IOReadableWritable> outputGate = gateIterator.next(); for (int i = 0; i < outputGate.getNumberOfInputChannels(); ++i) { inputChannelIDs.add(outputGate.getInputChannel(i).getID()); } } return Collections.unmodifiableSet(inputChannelIDs); }
/** * Constructs a runtime environment from a task deployment description. * * @param tdd the task deployment description * @param memoryManager the task manager's memory manager component * @param ioManager the task manager's I/O manager component * @param inputSplitProvider the input split provider for this environment * @throws Exception thrown if an error occurs while instantiating the invokable class */ @SuppressWarnings({"unchecked", "rawtypes"}) public RuntimeEnvironment( final TaskDeploymentDescriptor tdd, final MemoryManager memoryManager, final IOManager ioManager, final InputSplitProvider inputSplitProvider, AccumulatorProtocol accumulatorProtocolProxy) throws Exception { this.jobID = tdd.getJobID(); this.taskName = tdd.getTaskName(); this.invokableClass = tdd.getInvokableClass(); this.jobConfiguration = tdd.getJobConfiguration(); this.taskConfiguration = tdd.getTaskConfiguration(); this.indexInSubtaskGroup = tdd.getIndexInSubtaskGroup(); this.currentNumberOfSubtasks = tdd.getCurrentNumberOfSubtasks(); this.memoryManager = memoryManager; this.ioManager = ioManager; this.inputSplitProvider = inputSplitProvider; this.accumulatorProtocolProxy = accumulatorProtocolProxy; this.invokable = this.invokableClass.newInstance(); this.invokable.setEnvironment(this); this.invokable.registerInputOutput(); if (!this.unboundOutputGateIDs.isEmpty() && LOG.isErrorEnabled()) { LOG.error( "Inconsistency: " + this.unboundOutputGateIDs.size() + " unbound output gate IDs left"); } if (!this.unboundInputGateIDs.isEmpty() && LOG.isErrorEnabled()) { LOG.error( "Inconsistency: " + this.unboundInputGateIDs.size() + " unbound output gate IDs left"); } final int noogdd = tdd.getNumberOfOutputGateDescriptors(); for (int i = 0; i < noogdd; ++i) { final GateDeploymentDescriptor gdd = tdd.getOutputGateDescriptor(i); final OutputGate og = this.outputGates.get(i); final ChannelType channelType = gdd.getChannelType(); og.setChannelType(channelType); final int nocdd = gdd.getNumberOfChannelDescriptors(); for (int j = 0; j < nocdd; ++j) { final ChannelDeploymentDescriptor cdd = gdd.getChannelDescriptor(j); switch (channelType) { case NETWORK: og.createNetworkOutputChannel(og, cdd.getOutputChannelID(), cdd.getInputChannelID()); break; case INMEMORY: og.createInMemoryOutputChannel(og, cdd.getOutputChannelID(), cdd.getInputChannelID()); break; default: throw new IllegalStateException("Unknown channel type"); } } } final int noigdd = tdd.getNumberOfInputGateDescriptors(); for (int i = 0; i < noigdd; ++i) { final GateDeploymentDescriptor gdd = tdd.getInputGateDescriptor(i); final InputGate ig = this.inputGates.get(i); final ChannelType channelType = gdd.getChannelType(); ig.setChannelType(channelType); final int nicdd = gdd.getNumberOfChannelDescriptors(); for (int j = 0; j < nicdd; ++j) { final ChannelDeploymentDescriptor cdd = gdd.getChannelDescriptor(j); switch (channelType) { case NETWORK: ig.createNetworkInputChannel(ig, cdd.getInputChannelID(), cdd.getOutputChannelID()); break; case INMEMORY: ig.createInMemoryInputChannel(ig, cdd.getInputChannelID(), cdd.getOutputChannelID()); break; default: throw new IllegalStateException("Unknown channel type"); } } } }