mirror of https://github.com/apache/nifi.git
NIFI-1335: Allow UnpackContent to Handle application/x-tar mime.type
- Add additinal `mime.type` for tar archives (application/x-tar) - Update `UnpackContent` tests to verify auto-detect format property
This commit is contained in:
parent
a915534e23
commit
236a2b72b6
|
@ -175,6 +175,9 @@ public class UnpackContent extends AbstractProcessor {
|
|||
case "application/tar":
|
||||
packagingFormat = TAR_FORMAT;
|
||||
break;
|
||||
case "application/x-tar":
|
||||
packagingFormat = TAR_FORMAT;
|
||||
break;
|
||||
case "application/zip":
|
||||
packagingFormat = ZIP_FORMAT;
|
||||
break;
|
||||
|
|
|
@ -23,7 +23,9 @@ import java.io.IOException;
|
|||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
|
@ -38,17 +40,29 @@ public class TestUnpackContent {
|
|||
|
||||
@Test
|
||||
public void testTar() throws IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(new UnpackContent());
|
||||
runner.setProperty(UnpackContent.PACKAGING_FORMAT, UnpackContent.TAR_FORMAT);
|
||||
final TestRunner unpackRunner = TestRunners.newTestRunner(new UnpackContent());
|
||||
final TestRunner autoUnpackRunner = TestRunners.newTestRunner(new UnpackContent());
|
||||
unpackRunner.setProperty(UnpackContent.PACKAGING_FORMAT, UnpackContent.TAR_FORMAT);
|
||||
autoUnpackRunner.setProperty(UnpackContent.PACKAGING_FORMAT, UnpackContent.AUTO_DETECT_FORMAT);
|
||||
unpackRunner.enqueue(dataPath.resolve("data.tar"));
|
||||
Map<String, String> attributes = new HashMap<>(1);
|
||||
Map<String, String> attributes2 = new HashMap<>(1);
|
||||
attributes.put("mime.type", "application/x-tar");
|
||||
attributes2.put("mime.type", "application/tar");
|
||||
autoUnpackRunner.enqueue(dataPath.resolve("data.tar"), attributes);
|
||||
autoUnpackRunner.enqueue(dataPath.resolve("data.tar"), attributes2);
|
||||
unpackRunner.run();
|
||||
autoUnpackRunner.run(2);
|
||||
|
||||
runner.enqueue(dataPath.resolve("data.tar"));
|
||||
runner.run();
|
||||
unpackRunner.assertTransferCount(UnpackContent.REL_SUCCESS, 2);
|
||||
unpackRunner.assertTransferCount(UnpackContent.REL_ORIGINAL, 1);
|
||||
unpackRunner.assertTransferCount(UnpackContent.REL_FAILURE, 0);
|
||||
|
||||
runner.assertTransferCount(UnpackContent.REL_SUCCESS, 2);
|
||||
runner.assertTransferCount(UnpackContent.REL_ORIGINAL, 1);
|
||||
runner.assertTransferCount(UnpackContent.REL_FAILURE, 0);
|
||||
autoUnpackRunner.assertTransferCount(UnpackContent.REL_SUCCESS, 4);
|
||||
autoUnpackRunner.assertTransferCount(UnpackContent.REL_ORIGINAL, 2);
|
||||
autoUnpackRunner.assertTransferCount(UnpackContent.REL_FAILURE, 0);
|
||||
|
||||
final List<MockFlowFile> unpacked = runner.getFlowFilesForRelationship(UnpackContent.REL_SUCCESS);
|
||||
final List<MockFlowFile> unpacked = unpackRunner.getFlowFilesForRelationship(UnpackContent.REL_SUCCESS);
|
||||
for (final MockFlowFile flowFile : unpacked) {
|
||||
final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
|
||||
final String folder = flowFile.getAttribute(CoreAttributes.PATH.key());
|
||||
|
@ -61,17 +75,26 @@ public class TestUnpackContent {
|
|||
|
||||
@Test
|
||||
public void testZip() throws IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(new UnpackContent());
|
||||
runner.setProperty(UnpackContent.PACKAGING_FORMAT, UnpackContent.ZIP_FORMAT);
|
||||
runner.enqueue(dataPath.resolve("data.zip"));
|
||||
final TestRunner unpackRunner = TestRunners.newTestRunner(new UnpackContent());
|
||||
final TestRunner autoUnpackRunner = TestRunners.newTestRunner(new UnpackContent());
|
||||
unpackRunner.setProperty(UnpackContent.PACKAGING_FORMAT, UnpackContent.ZIP_FORMAT);
|
||||
autoUnpackRunner.setProperty(UnpackContent.PACKAGING_FORMAT, UnpackContent.AUTO_DETECT_FORMAT);
|
||||
unpackRunner.enqueue(dataPath.resolve("data.zip"));
|
||||
Map<String, String> attributes = new HashMap<>(1);
|
||||
attributes.put("mime.type", "application/zip");
|
||||
autoUnpackRunner.enqueue(dataPath.resolve("data.zip"), attributes);
|
||||
unpackRunner.run();
|
||||
autoUnpackRunner.run();
|
||||
|
||||
runner.run();
|
||||
unpackRunner.assertTransferCount(UnpackContent.REL_SUCCESS, 2);
|
||||
unpackRunner.assertTransferCount(UnpackContent.REL_ORIGINAL, 1);
|
||||
unpackRunner.assertTransferCount(UnpackContent.REL_FAILURE, 0);
|
||||
|
||||
runner.assertTransferCount(UnpackContent.REL_SUCCESS, 2);
|
||||
runner.assertTransferCount(UnpackContent.REL_ORIGINAL, 1);
|
||||
runner.assertTransferCount(UnpackContent.REL_FAILURE, 0);
|
||||
autoUnpackRunner.assertTransferCount(UnpackContent.REL_SUCCESS, 2);
|
||||
autoUnpackRunner.assertTransferCount(UnpackContent.REL_ORIGINAL, 1);
|
||||
autoUnpackRunner.assertTransferCount(UnpackContent.REL_FAILURE, 0);
|
||||
|
||||
final List<MockFlowFile> unpacked = runner.getFlowFilesForRelationship(UnpackContent.REL_SUCCESS);
|
||||
final List<MockFlowFile> unpacked = unpackRunner.getFlowFilesForRelationship(UnpackContent.REL_SUCCESS);
|
||||
for (final MockFlowFile flowFile : unpacked) {
|
||||
final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
|
||||
final String folder = flowFile.getAttribute(CoreAttributes.PATH.key());
|
||||
|
|
Loading…
Reference in New Issue