NIFI-3218: throw exception in MockProcessSession when transferring file to self

This commit is contained in:
m-hogue 2017-07-06 12:10:09 -04:00 committed by Joe Skora
parent 0fd51b4d12
commit 58f60b3cf0
2 changed files with 16 additions and 0 deletions

View File

@ -756,6 +756,13 @@ public class MockProcessSession implements ProcessSession {
throw new IllegalArgumentException("I only accept MockFlowFile");
}
// if the flowfile provided was created in this session (i.e. it's in currentVersions),
// then throw an exception indicating that you can't transfer flowfiles back to self.
// this mimics the behavior of StandardProcessSession
if(currentVersions.get(flowFile.getId()) != null) {
throw new IllegalArgumentException("Cannot transfer FlowFiles that are created in this Session back to self");
}
final MockFlowFile mockFlowFile = (MockFlowFile) flowFile;
beingProcessed.remove(flowFile.getId());
processorQueue.offer(mockFlowFile);

View File

@ -80,6 +80,15 @@ public class TestMockProcessSession {
}
@Test(expected = IllegalArgumentException.class)
public void testRejectTransferNewlyCreatedFileToSelf() {
final Processor processor = new PoorlyBehavedProcessor();
final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, new AtomicLong(0L)), processor);
final FlowFile ff1 = session.createFlowFile("hello, world".getBytes());
// this should throw an exception because we shouldn't allow a newly created flowfile to get routed back to self
session.transfer(ff1);
}
protected static class PoorlyBehavedProcessor extends AbstractProcessor {
private static final Relationship REL_FAILURE = new Relationship.Builder()