NIFI-4686 MockFlowFile now includes penalized status when copying. This closes #2438.

Signed-off-by: Mark Payne <markap14@hotmail.com>
This commit is contained in:
jlwt90 2018-01-28 11:37:50 +09:00 committed by Mark Payne
parent 8bdc2910e1
commit d8cfb8e6c5
3 changed files with 15 additions and 0 deletions

View File

@ -94,6 +94,8 @@ public class MockFlowFile implements FlowFileRecord {
final byte[] dataToCopy = ((MockFlowFile) toCopy).data;
this.data = new byte[dataToCopy.length];
System.arraycopy(dataToCopy, 0, this.data, 0, dataToCopy.length);
this.penalized = toCopy.isPenalized();
}
void setPenalized() {

View File

@ -89,6 +89,18 @@ public class TestMockProcessSession {
session.transfer(ff1);
}
@Test
public void testKeepPenalizedStatusAfterPuttingAttribute(){
final Processor processor = new PoorlyBehavedProcessor();
final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, new AtomicLong(0L)), processor);
FlowFile ff1 = session.createFlowFile("hello, world".getBytes());
ff1 = session.penalize(ff1);
assertEquals(true, ff1.isPenalized());
ff1 = session.putAttribute(ff1, "hello", "world");
// adding attribute to flow file should not override the original penalized status
assertEquals(true, ff1.isPenalized());
}
protected static class PoorlyBehavedProcessor extends AbstractProcessor {
private static final Relationship REL_FAILURE = new Relationship.Builder()

View File

@ -407,6 +407,7 @@ public class TestGetCouchbaseKey {
MockFlowFile orgFile = testRunner.getFlowFilesForRelationship(REL_RETRY).get(0);
orgFile.assertContentEquals(inputFileDataStr);
orgFile.assertAttributeEquals(Exception.key(), exception.getClass().getName());
Assert.assertEquals(true, orgFile.isPenalized());
}
@Test