Fixes NIFI-1220. This closes #133. MockProcessSession returns a new FlowFile from its `penalty` method instead of mutating then returning the given FlowFile

Signed-off-by: joewitt <joewitt@apache.org>
This commit is contained in:
Johnathan Gilday 2015-11-25 13:34:06 -05:00 committed by joewitt
parent 0609a84fa2
commit fd35b8ffd7
2 changed files with 62 additions and 2 deletions

View File

@ -1010,8 +1010,10 @@ public class MockProcessSession implements ProcessSession {
public MockFlowFile penalize(final FlowFile flowFile) {
validateState(flowFile);
final MockFlowFile mockFlowFile = (MockFlowFile) flowFile;
mockFlowFile.setPenalized();
return mockFlowFile;
final MockFlowFile newFlowFile = new MockFlowFile(mockFlowFile.getId(), flowFile);
currentVersions.put(newFlowFile.getId(), newFlowFile);
newFlowFile.setPenalized();
return newFlowFile;
}
public byte[] getContentAsByteArray(final MockFlowFile flowFile) {

View File

@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.util;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.junit.Test;
import java.util.Collections;
import java.util.Set;
public class TestMockProcessSession {
@Test(expected = AssertionError.class)
public void testPenalizeFlowFileFromProcessor() {
TestRunners.newTestRunner(PoorlyBehavedProcessor.class).run();
}
protected static class PoorlyBehavedProcessor extends AbstractProcessor {
private static final Relationship REL_FAILURE = new Relationship.Builder()
.name("failure")
.build();
private final Set<Relationship> relationships = Collections.singleton(REL_FAILURE);
@Override
public Set<Relationship> getRelationships() {
return relationships;
}
@Override
public void onTrigger(final ProcessContext ctx, final ProcessSession session) throws ProcessException {
final FlowFile file = session.create();
session.penalize(file);
session.transfer(file, REL_FAILURE);
}
}
}