From 947c7b64e09e8f0e6e16cc56df9f15d7a0fc6d6d Mon Sep 17 00:00:00 2001 From: Tamas Palfy Date: Wed, 1 Jul 2020 16:54:46 +0200 Subject: [PATCH] NIFI-7594 In HandleHttpRequest deleting multipart file resources after processing. This closes #4379. Signed-off-by: Peter Turcsanyi --- .../nifi/processors/standard/HandleHttpRequest.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java index bdc9a99705cc..86ee12643fdf 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java @@ -640,8 +640,9 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final int readBufferSize = context.getProperty(MULTIPART_READ_BUFFER_SIZE).asDataSize(DataUnit.B).intValue(); String tempDir = System.getProperty("java.io.tmpdir"); request.setAttribute(Request.MULTIPART_CONFIG_ELEMENT, new MultipartConfigElement(tempDir, requestMaxSize, requestMaxSize, readBufferSize)); + List parts = null; try { - List parts = ImmutableList.copyOf(request.getParts()); + parts = ImmutableList.copyOf(request.getParts()); int allPartsCount = parts.size(); final String contextIdentifier = UUID.randomUUID().toString(); for (int i = 0; i < allPartsCount; i++) { @@ -666,6 +667,16 @@ public void onTrigger(final ProcessContext context, final ProcessSession session } catch (IOException | ServletException | IllegalStateException e) { handleFlowContentStreamingError(session, container, request, Optional.absent(), e); return; + } finally { + if (parts != null) { + for (Part part : parts) { + try { + part.delete(); + } catch (Exception e) { + getLogger().error("Couldn't delete underlying storage for {}", new Object[]{part}, e); + } + } + } } } else { FlowFile flowFile = session.create();