From 180a188096905ec3c599f45a4c0235778951fb7c Mon Sep 17 00:00:00 2001 From: Bryan Bende Date: Tue, 3 Sep 2019 10:50:54 -0400 Subject: [PATCH 01/11] NIFI-6028 Protect against null VersionedComponent when finding relevant process group This closes #3686. Signed-off-by: Mark Payne --- .../src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index 8e94a5b25a91..d2d331d6b4c7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -2472,7 +2472,8 @@ public Set createComponentDifferenceDtos(final FlowCompa continue; } - final VersionedProcessGroup relevantProcessGroup = versionedGroups.get(difference.getComponentA().getGroupIdentifier()); + final VersionedComponent componentA = difference.getComponentA(); + final VersionedProcessGroup relevantProcessGroup = componentA == null ? null : versionedGroups.get(componentA.getGroupIdentifier()); if (relevantProcessGroup != null && FlowDifferenceFilters.isNewRelationshipAutoTerminatedAndDefaulted(difference, relevantProcessGroup, flowManager)) { continue; } From 07e1c11520aa5490c6d10afae17ab74b2fab0d1c Mon Sep 17 00:00:00 2001 From: Matt Gilman Date: Tue, 3 Sep 2019 11:52:45 -0400 Subject: [PATCH 02/11] NIFI-6608: - Conditionally support parameters in controller service configuration. This closes #3688 Signed-off-by: Scott Aslan --- .../js/nf/canvas/nf-controller-service.js | 91 +++++++++++-------- 1 file changed, 53 insertions(+), 38 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js index 9017ce471b10..6814edcd1fe3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js @@ -1509,6 +1509,52 @@ }).fail(nfErrorHandler.handleAjaxError); }; + /** + * Gets the parameters for the specified property descriptor and group. + * + * @param propertyDescriptor The property descriptor in question + * @param groupId The group in question + * @returns {deferred} + */ + var getParameters = function (propertyDescriptor, groupId) { + return $.Deferred(function (deferred) { + if (nfCommon.isDefinedAndNotNull(groupId)) { + var parameterContextId; + + // attempt to identify the parameter context id, conditional based on whether + // the user is configuring the current process group + if (groupId === nfCanvasUtils.getGroupId()) { + parameterContextId = nfCanvasUtils.getParameterContextId(); + } else { + var parentProcessGroup = nfCanvasUtils.getComponentByType('ProcessGroup').get(groupId); + parameterContextId = parentProcessGroup.parameterContextId; + } + + if (nfCommon.isDefinedAndNotNull(parameterContextId)) { + $.ajax({ + type: 'GET', + url: '../nifi-api/parameter-contexts/' + parameterContextId, + dataType: 'json' + }).done(function (response) { + var sensitive = nfCommon.isSensitiveProperty(propertyDescriptor); + + deferred.resolve(response.component.parameters.map(function (parameterEntity) { + return parameterEntity.parameter; + }).filter(function (parameter) { + return parameter.sensitive === sensitive; + })); + }).fail(function () { + deferred.resolve([]); + }); + } else { + deferred.resolve([]); + } + } else { + deferred.resolve([]); + } + }).promise(); + }; + /** * Goes to a service configuration from the property table. * @@ -1800,50 +1846,19 @@ $('#controller-service-configuration .controller-service-read-only').hide(); $('#controller-service-configuration .controller-service-editable').show(); + // conditionally get the parameter deferred function + var getParameterDeferred = null; + if (nfCommon.isDefinedAndNotNull(controllerServiceEntity.parentGroupId)) { + getParameterDeferred = getParameters; + } + // initialize the property table $('#controller-service-properties').propertytable('destroy').propertytable({ readOnly: false, supportsGoTo: true, dialogContainer: '#new-controller-service-property-container', descriptorDeferred: getControllerServicePropertyDescriptor, - parameterDeferred: function (propertyDescriptor, groupId) { - return $.Deferred(function (deferred) { - if (nfCommon.isDefinedAndNotNull(groupId)) { - var parameterContextId; - - // attempt to identify the parameter context id, conditional based on whether - // the user is configuring the current process group - if (groupId === nfCanvasUtils.getGroupId()) { - parameterContextId = nfCanvasUtils.getParameterContextId(); - } else { - var parentProcessGroup = nfCanvasUtils.getComponentByType('ProcessGroup').get(groupId); - parameterContextId = parentProcessGroup.parameterContextId; - } - - if (nfCommon.isDefinedAndNotNull(parameterContextId)) { - $.ajax({ - type: 'GET', - url: '../nifi-api/parameter-contexts/' + parameterContextId, - dataType: 'json' - }).done(function (response) { - var sensitive = nfCommon.isSensitiveProperty(propertyDescriptor); - - deferred.resolve(response.component.parameters.map(function (parameterEntity) { - return parameterEntity.parameter; - }).filter(function (parameter) { - return parameter.sensitive === sensitive; - })); - }).fail(function () { - deferred.resolve([]); - }); - } else { - deferred.resolve([]); - } - } else { - deferred.resolve([]); - } - }).promise(); - }, + parameterDeferred: getParameterDeferred, controllerServiceCreatedDeferred: function (response) { var controllerServicesUri; From 4851e00f1da758468984c198b8e9f49a006d5f0c Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Tue, 3 Sep 2019 12:15:12 -0400 Subject: [PATCH 03/11] NIFI-6607: Ensure that when creating ParameterContextUpdateRequestDTO that we populate the AffectedComponentEntities with the most up-to-date version of the component This closes #3689 --- .../apache/nifi/web/NiFiServiceFacade.java | 7 ++++ .../nifi/web/StandardNiFiServiceFacade.java | 40 +++++++++++++++++++ .../web/api/ParameterContextResource.java | 5 ++- 3 files changed, 50 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java index f62bec217907..a349a391a58c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java @@ -1573,6 +1573,13 @@ ProcessGroupEntity updateProcessGroupContents(Revision revision, String groupId, */ Set getComponentsAffectedByParameterContextUpdate(ParameterContextDTO parameterContextDto); + /** + * Returns an up-to-date representation of the component that is referenced by the given affected component + * @param affectedComponent the affected component + * @return an up-to-date representation of the affected component + */ + AffectedComponentEntity getUpdatedAffectedComponentEntity(AffectedComponentEntity affectedComponent); + /** * Returns a Set representing all Processors that reference any Parameters and that belong to the group with the given ID * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index 03cdc317cb0d..cbc5ec23e63d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -1194,6 +1194,46 @@ private Set getComponentsReferencingParameter(final Str return dtoFactory.createAffectedComponentEntities(affectedComponents, revisionManager); } + @Override + public AffectedComponentEntity getUpdatedAffectedComponentEntity(final AffectedComponentEntity affectedComponent) { + final AffectedComponentDTO dto = affectedComponent.getComponent(); + if (dto == null) { + return affectedComponent; + } + + final String groupId = affectedComponent.getProcessGroup().getId(); + final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId); + + final String componentType = dto.getReferenceType(); + if (AffectedComponentDTO.COMPONENT_TYPE_CONTROLLER_SERVICE.equals(componentType)) { + final ControllerServiceNode serviceNode = processGroup.getControllerService(dto.getId()); + return dtoFactory.createAffectedComponentEntity(serviceNode, revisionManager); + } else if (AffectedComponentDTO.COMPONENT_TYPE_PROCESSOR.equals(componentType)) { + final ProcessorNode processorNode = processGroup.getProcessor(dto.getId()); + return dtoFactory.createAffectedComponentEntity(processorNode, revisionManager); + } else if (AffectedComponentDTO.COMPONENT_TYPE_INPUT_PORT.equals(componentType)) { + final Port inputPort = processGroup.getInputPort(dto.getId()); + final PortEntity portEntity = createInputPortEntity(inputPort); + return dtoFactory.createAffectedComponentEntity(portEntity, AffectedComponentDTO.COMPONENT_TYPE_INPUT_PORT); + } else if (AffectedComponentDTO.COMPONENT_TYPE_OUTPUT_PORT.equals(componentType)) { + final Port outputPort = processGroup.getOutputPort(dto.getId()); + final PortEntity portEntity = createOutputPortEntity(outputPort); + return dtoFactory.createAffectedComponentEntity(portEntity, AffectedComponentDTO.COMPONENT_TYPE_OUTPUT_PORT); + } else if (AffectedComponentDTO.COMPONENT_TYPE_REMOTE_INPUT_PORT.equals(componentType)) { + final RemoteGroupPort remoteGroupPort = processGroup.findRemoteGroupPort(dto.getId()); + final RemoteProcessGroupEntity rpgEntity = createRemoteGroupEntity(remoteGroupPort.getRemoteProcessGroup(), NiFiUserUtils.getNiFiUser()); + final RemoteProcessGroupPortDTO remotePortDto = dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort); + return dtoFactory.createAffectedComponentEntity(remotePortDto, AffectedComponentDTO.COMPONENT_TYPE_REMOTE_INPUT_PORT, rpgEntity); + } else if (AffectedComponentDTO.COMPONENT_TYPE_REMOTE_OUTPUT_PORT.equals(componentType)) { + final RemoteGroupPort remoteGroupPort = processGroup.findRemoteGroupPort(dto.getId()); + final RemoteProcessGroupEntity rpgEntity = createRemoteGroupEntity(remoteGroupPort.getRemoteProcessGroup(), NiFiUserUtils.getNiFiUser()); + final RemoteProcessGroupPortDTO remotePortDto = dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort); + return dtoFactory.createAffectedComponentEntity(remotePortDto, AffectedComponentDTO.COMPONENT_TYPE_REMOTE_OUTPUT_PORT, rpgEntity); + } + + return affectedComponent; + } + @Override public Set getComponentsAffectedByParameterContextUpdate(final ParameterContextDTO parameterContextDto) { return getComponentsAffectedByParameterContextUpdate(parameterContextDto, true); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java index 63fcad606217..5c8fd6a9bf42 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java @@ -1155,13 +1155,14 @@ private ParameterContextUpdateRequestEntity createUpdateRequestEntity(final Asyn updateRequestDto.setUpdateSteps(updateSteps); final ParameterContextEntity initialRequest = asyncRequest.getRequest(); + + // The AffectedComponentEntity itself does not evaluate equality based on component information. As a result, we want to de-dupe the entities based on their identifiers. final Map affectedComponents = new HashMap<>(); for (final ParameterEntity entity : initialRequest.getComponent().getParameters()) { for (final AffectedComponentEntity affectedComponentEntity : entity.getParameter().getReferencingComponents()) { - affectedComponents.put(affectedComponentEntity.getId(), affectedComponentEntity); + affectedComponents.put(affectedComponentEntity.getId(), serviceFacade.getUpdatedAffectedComponentEntity(affectedComponentEntity)); } } - updateRequestDto.setReferencingComponents(new HashSet<>(affectedComponents.values())); // Populate the Affected Components From 459ef24e99dc6fcdea53e6b264adba048b7c0592 Mon Sep 17 00:00:00 2001 From: Scott Aslan Date: Tue, 3 Sep 2019 12:54:10 -0400 Subject: [PATCH 04/11] [NIFI-6606] disable PC for which the user does not have read permissions when configuring a PG. Also, sort the list of PCs for the combo dropdown in the PG configuration This closes #3690 --- .../canvas/nf-process-group-configuration.js | 36 ++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-process-group-configuration.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-process-group-configuration.js index 8bf333da51ac..43e84f28d0e6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-process-group-configuration.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-process-group-configuration.js @@ -284,7 +284,40 @@ text: 'No parameter context', value: null }]; - parameterContexts.forEach(function (parameterContext) { + + var authorizedParameterContexts = parameterContexts.filter(function (parameterContext) { + return parameterContext.permissions.canRead; + }); + + var unauthorizedParameterContexts = parameterContexts.filter(function (parameterContext) { + return !parameterContext.permissions.canRead; + }); + + //sort alphabetically + var sortedAuthorizedParameterContexts = authorizedParameterContexts.sort(function (a, b) { + if (a.component.name < b.component.name) { + return -1; + } + if (a.component.name > b.component.name) { + return 1; + } + return 0; + }); + + //sort alphabetically + var sortedUnauthorizedParameterContexts = unauthorizedParameterContexts.sort(function (a, b) { + if (a.id < b.id) { + return -1; + } + if (a.id > b.id) { + return 1; + } + return 0; + }); + + var sortedParameterContexts = sortedAuthorizedParameterContexts.concat(sortedUnauthorizedParameterContexts); + + sortedParameterContexts.forEach(function (parameterContext) { var option; if (parameterContext.permissions.canRead) { option = { @@ -294,6 +327,7 @@ }; } else { option = { + 'disabled': true, 'text': parameterContext.id, 'value': parameterContext.id } From a2edc2aa870ee465b70d9b5e319548e478e481a9 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Tue, 3 Sep 2019 11:19:47 -0400 Subject: [PATCH 05/11] NIFI-6025: Fixed bug that caused Flows to indicate that they have been locally modified (when they hadn't been) when running against versions of the registry older than 0.5.0 This closes #3687. Signed-off-by: Bryan Bende --- .../apache/nifi/groups/StandardProcessGroup.java | 1 + .../apache/nifi/util/FlowDifferenceFilters.java | 16 ++++++++++++++++ .../nifi/web/StandardNiFiServiceFacade.java | 1 + .../org/apache/nifi/web/api/dto/DtoFactory.java | 7 +++++++ 4 files changed, 25 insertions(+) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 858d82656b42..52565521f89f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -4737,6 +4737,7 @@ private Set getModifications() { .filter(FlowDifferenceFilters.FILTER_IGNORABLE_VERSIONED_FLOW_COORDINATE_CHANGES) .filter(diff -> !FlowDifferenceFilters.isNewPropertyWithDefaultValue(diff, flowManager)) .filter(diff -> !FlowDifferenceFilters.isNewRelationshipAutoTerminatedAndDefaulted(diff, versionedGroup, flowManager)) + .filter(diff -> !FlowDifferenceFilters.isScheduledStateNew(diff)) .collect(Collectors.toCollection(HashSet::new)); LOG.debug("There are {} differences between this Local Flow and the Versioned Flow: {}", differences.size(), differences); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java index d594f5f33bf4..bc64d5ec4a06 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java @@ -23,6 +23,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.processor.Relationship; import org.apache.nifi.registry.flow.ComponentType; +import org.apache.nifi.registry.flow.ScheduledState; import org.apache.nifi.registry.flow.VersionedComponent; import org.apache.nifi.registry.flow.VersionedConnection; import org.apache.nifi.registry.flow.VersionedFlowCoordinates; @@ -163,6 +164,21 @@ private static boolean isNewPropertyWithDefaultValue(final FlowDifference fd, fi return false; } + public static boolean isScheduledStateNew(final FlowDifference fd) { + if (fd.getDifferenceType() != DifferenceType.SCHEDULED_STATE_CHANGED) { + return false; + } + + // If Scheduled State transitions from null to ENABLED or ENABLED to null, consider it a "new" scheduled state. + if (fd.getValueA() == null && ScheduledState.ENABLED.equals(fd.getValueB())) { + return true; + } + if (fd.getValueB() == null && "ENABLED".equals(fd.getValueA())) { + return true; + } + + return false; + } public static boolean isNewRelationshipAutoTerminatedAndDefaulted(final FlowDifference fd, final VersionedProcessGroup processGroup, final FlowManager flowManager) { if (fd.getDifferenceType() != DifferenceType.AUTO_TERMINATED_RELATIONSHIPS_CHANGED) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index cbc5ec23e63d..8ebeec97d3bc 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -4569,6 +4569,7 @@ public Set getComponentsAffectedByVersionChange(final S .filter(FlowDifferenceFilters.FILTER_IGNORABLE_VERSIONED_FLOW_COORDINATE_CHANGES) .filter(diff -> !FlowDifferenceFilters.isNewPropertyWithDefaultValue(diff, flowManager)) .filter(diff -> !FlowDifferenceFilters.isNewRelationshipAutoTerminatedAndDefaulted(diff, proposedFlow.getContents(), flowManager)) + .filter(diff -> !FlowDifferenceFilters.isScheduledStateNew(diff)) .map(difference -> { final VersionedComponent localComponent = difference.getComponentA(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index d2d331d6b4c7..3be7ae765d37 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -2454,6 +2454,13 @@ public Set createComponentDifferenceDtos(final FlowCompa continue; } + // Ignore differences that are the result of the Versioned Flow not having a Scheduled State and the newer flow being "ENABLED". We do this because + // Scheduled State was not always part of the Versioned Flow - it was always assumed to be ENABLED. We don't want flows that were previously stored in this + // format to now be considered different than the local flow. + if (FlowDifferenceFilters.isScheduledStateNew(difference)) { + continue; + } + // Ignore differences for adding remote ports if (FlowDifferenceFilters.isAddedOrRemovedRemotePort(difference)) { continue; From 4c6c1cbb14aaedda649981f76d71ced2e8a36eac Mon Sep 17 00:00:00 2001 From: Alessandro D'Armiento Date: Sat, 17 Aug 2019 18:30:24 +0200 Subject: [PATCH 06/11] NIFI-6536 EncryptContent accepts configurable PGP symmetric cipher Additional test case in TestEncryptContent Apply suggestions from code review nit picking Apply suggestions from code review nitpicking from mgaido91 Small fixes Additional test case to check the provided cipher is actually used by inferring it from the ciphertext Updated `EncryptContent` to use a pre-defined list of allowable PGP ciphers Updated `EncryptContent` to check the validity of `PGP_SYMMETRIC_ENCRYPTION_CIPHER` Updated test cases Minor fixes NIFI-6536 Fixed typo and strengthened validity checking for PGP symmetric cipher. This closes #3664. Co-Authored-By: Marco Gaido Signed-off-by: Andy LoPresto --- .../processors/standard/EncryptContent.java | 67 +++++++-- .../util/crypto/OpenPGPKeyBasedEncryptor.java | 15 +- .../crypto/OpenPGPPasswordBasedEncryptor.java | 13 +- .../standard/TestEncryptContent.java | 131 ++++++++++++++++++ .../crypto/OpenPGPKeyBasedEncryptorTest.java | 128 +++++++++-------- .../OpenPGPPasswordBasedEncryptorTest.java | 107 ++++++++------ 6 files changed, 340 insertions(+), 121 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java index a3249ba3974c..b59b5faab293 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java @@ -63,6 +63,7 @@ import org.apache.nifi.security.util.crypto.PasswordBasedEncryptor; import org.apache.nifi.util.StopWatch; import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.bouncycastle.openpgp.PGPEncryptedData; @EventDriven @SideEffectFree @@ -138,6 +139,18 @@ public class EncryptContent extends AbstractProcessor { .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .sensitive(true) .build(); + + public static final PropertyDescriptor PGP_SYMMETRIC_ENCRYPTION_CIPHER = new PropertyDescriptor.Builder() + .name("pgp-symmetric-cipher") + .displayName("PGP Symmetric Cipher") + .description("When using PGP encryption, this is the symmetric cipher to be used. This property is ignored if " + + "Encryption Algorithm is not PGP or PGP-ASCII-ARMOR\nNote that the provided cipher is only used during" + + "the encryption phase, while it is inferred from the ciphertext in the decryption phase") + .required(false) + .allowableValues(buildPGPSymmetricCipherAllowableValues()) + .defaultValue(String.valueOf(PGPEncryptedData.AES_128)) + .build(); + public static final PropertyDescriptor RAW_KEY_HEX = new PropertyDescriptor.Builder() .name("raw-key-hex") .displayName("Raw Key (hexadecimal)") @@ -202,6 +215,23 @@ private static AllowableValue buildDefaultWeakCryptoAllowableValue() { "if unsafe combinations of encryption algorithms and passwords are provided on a JVM with limited strength crypto. To fix this, see the Admin Guide."); } + private static AllowableValue[] buildPGPSymmetricCipherAllowableValues() { + // Allowed values are inferred from SymmetricKeyAlgorithmTags. Note that NULL and SAFER cipher are not supported and therefore not listed + return new AllowableValue[] { + new AllowableValue("1", "IDEA"), + new AllowableValue("2", "TRIPLE_DES"), + new AllowableValue("3", "CAST5"), + new AllowableValue("4", "BLOWFISH"), + new AllowableValue("6", "DES"), + new AllowableValue("7", "AES_128"), + new AllowableValue("8", "AES_192"), + new AllowableValue("9", "AES_256"), + new AllowableValue("10", "TWOFISH"), + new AllowableValue("11", "CAMELLIA_128"), + new AllowableValue("12", "CAMELLIA_192"), + new AllowableValue("13", "CAMELLIA_256") }; + } + @Override protected void init(final ProcessorInitializationContext context) { final List properties = new ArrayList<>(); @@ -215,6 +245,7 @@ protected void init(final ProcessorInitializationContext context) { properties.add(PUBLIC_KEY_USERID); properties.add(PRIVATE_KEYRING); properties.add(PRIVATE_KEYRING_PASSPHRASE); + properties.add(PGP_SYMMETRIC_ENCRYPTION_CIPHER); this.properties = Collections.unmodifiableList(properties); final Set relationships = new HashSet<>(); @@ -256,7 +287,9 @@ protected Collection customValidate(final ValidationContext co final String publicUserId = context.getProperty(PUBLIC_KEY_USERID).getValue(); final String privateKeyring = context.getProperty(PRIVATE_KEYRING).getValue(); final String privateKeyringPassphrase = context.getProperty(PRIVATE_KEYRING_PASSPHRASE).evaluateAttributeExpressions().getValue(); - validationResults.addAll(validatePGP(encryptionMethod, password, encrypt, publicKeyring, publicUserId, privateKeyring, privateKeyringPassphrase)); + final Integer cipher = context.getProperty(PGP_SYMMETRIC_ENCRYPTION_CIPHER).asInteger(); + validationResults.addAll(validatePGP(encryptionMethod, password, encrypt, publicKeyring, publicUserId, + privateKeyring, privateKeyringPassphrase, cipher)); } else { // Not PGP if (encryptionMethod.isKeyedCipher()) { // Raw key validationResults.addAll(validateKeyed(encryptionMethod, kdf, keyHex)); @@ -268,10 +301,28 @@ protected Collection customValidate(final ValidationContext co return validationResults; } - private List validatePGP(EncryptionMethod encryptionMethod, String password, boolean encrypt, String publicKeyring, String publicUserId, String privateKeyring, - String privateKeyringPassphrase) { + /** + * Returns true if the integer value provided maps to a valid {@code cipher} as contained in the {@code PGP_SYMMETRIC_ENCRYPTION_CIPHER}. + * + * @param cipher an integer indicating a particular cipher + * @return true if the cipher is supported + */ + private static boolean isValidCipher(int cipher) { + return PGP_SYMMETRIC_ENCRYPTION_CIPHER.getAllowableValues().stream().anyMatch(av -> av.getValue().equals(String.valueOf(cipher))); + } + + private List validatePGP(EncryptionMethod encryptionMethod, String password, boolean encrypt, + String publicKeyring, String publicUserId, String privateKeyring, + String privateKeyringPassphrase, int cipher) { List validationResults = new ArrayList<>(); + if(encrypt && password != null && !isValidCipher(cipher)) { + validationResults.add(new ValidationResult.Builder().subject(PGP_SYMMETRIC_ENCRYPTION_CIPHER.getDisplayName()) + .explanation("When performing an encryption with " + encryptionMethod.getAlgorithm() + " and a symmetric " + + PASSWORD.getDisplayName() + ", a" + PGP_SYMMETRIC_ENCRYPTION_CIPHER.getDisplayName() + " is required") + .build()); + } + if (password == null) { if (encrypt) { // If encrypting without a password, require both public-keyring-file and public-key-user-id @@ -463,6 +514,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final EncryptionMethod encryptionMethod = EncryptionMethod.valueOf(method); final String providerName = encryptionMethod.getProvider(); final String algorithm = encryptionMethod.getAlgorithm(); + final Integer pgpCipher = context.getProperty(PGP_SYMMETRIC_ENCRYPTION_CIPHER).asInteger(); final String password = context.getProperty(PASSWORD).getValue(); final KeyDerivationFunction kdf = KeyDerivationFunction.valueOf(context.getProperty(KEY_DERIVATION_FUNCTION).getValue()); final boolean encrypt = context.getProperty(MODE).getValue().equalsIgnoreCase(ENCRYPT_MODE); @@ -476,14 +528,13 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final String privateKeyring = context.getProperty(PRIVATE_KEYRING).getValue(); if (encrypt && publicKeyring != null) { final String publicUserId = context.getProperty(PUBLIC_KEY_USERID).getValue(); - encryptor = new OpenPGPKeyBasedEncryptor(algorithm, providerName, publicKeyring, publicUserId, null, filename); + encryptor = new OpenPGPKeyBasedEncryptor(algorithm, pgpCipher, providerName, publicKeyring, publicUserId, null, filename); } else if (!encrypt && privateKeyring != null) { final char[] keyringPassphrase = context.getProperty(PRIVATE_KEYRING_PASSPHRASE).evaluateAttributeExpressions().getValue().toCharArray(); - encryptor = new OpenPGPKeyBasedEncryptor(algorithm, providerName, privateKeyring, null, keyringPassphrase, - filename); + encryptor = new OpenPGPKeyBasedEncryptor(algorithm, pgpCipher, providerName, privateKeyring, null, keyringPassphrase, filename); } else { final char[] passphrase = Normalizer.normalize(password, Normalizer.Form.NFC).toCharArray(); - encryptor = new OpenPGPPasswordBasedEncryptor(algorithm, providerName, passphrase, filename); + encryptor = new OpenPGPPasswordBasedEncryptor(algorithm, pgpCipher, providerName, passphrase, filename); } } else if (kdf.equals(KeyDerivationFunction.NONE)) { // Raw key final String keyHex = context.getProperty(RAW_KEY_HEX).getValue(); @@ -524,4 +575,4 @@ public interface Encryptor { StreamCallback getDecryptionCallback() throws Exception; } -} \ No newline at end of file +} diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptor.java index 6b6c2fc9557a..6b5e78073d32 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptor.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptor.java @@ -35,7 +35,6 @@ import org.bouncycastle.bcpg.ArmoredOutputStream; import org.bouncycastle.openpgp.PGPCompressedData; import org.bouncycastle.openpgp.PGPCompressedDataGenerator; -import org.bouncycastle.openpgp.PGPEncryptedData; import org.bouncycastle.openpgp.PGPEncryptedDataGenerator; import org.bouncycastle.openpgp.PGPEncryptedDataList; import org.bouncycastle.openpgp.PGPException; @@ -67,6 +66,7 @@ public class OpenPGPKeyBasedEncryptor implements Encryptor { private static final Logger logger = LoggerFactory.getLogger(OpenPGPPasswordBasedEncryptor.class); private String algorithm; + private Integer cipher; private String provider; // TODO: This can hold either the secret or public keyring path private String keyring; @@ -74,8 +74,10 @@ public class OpenPGPKeyBasedEncryptor implements Encryptor { private char[] passphrase; private String filename; - public OpenPGPKeyBasedEncryptor(final String algorithm, final String provider, final String keyring, final String userId, final char[] passphrase, final String filename) { + public OpenPGPKeyBasedEncryptor(final String algorithm, final Integer cipher, final String provider, final String keyring, + final String userId, final char[] passphrase, final String filename) { this.algorithm = algorithm; + this.cipher = cipher; this.provider = provider; this.keyring = keyring; this.userId = userId; @@ -85,7 +87,7 @@ public OpenPGPKeyBasedEncryptor(final String algorithm, final String provider, f @Override public StreamCallback getEncryptionCallback() throws Exception { - return new OpenPGPEncryptCallback(algorithm, provider, keyring, userId, filename); + return new OpenPGPEncryptCallback(algorithm, cipher, provider, keyring, userId, filename); } @Override @@ -314,13 +316,15 @@ public void process(InputStream in, OutputStream out) throws IOException { private static class OpenPGPEncryptCallback implements StreamCallback { private String algorithm; + private Integer cipher; private String provider; private String publicKeyring; private String userId; private String filename; - OpenPGPEncryptCallback(final String algorithm, final String provider, final String keyring, final String userId, final String filename) { + OpenPGPEncryptCallback(final String algorithm, final Integer cipher, final String provider, final String keyring, final String userId, final String filename) { this.algorithm = algorithm; + this.cipher = cipher; this.provider = provider; this.publicKeyring = keyring; this.userId = userId; @@ -345,9 +349,8 @@ public void process(InputStream in, OutputStream out) throws IOException { } try { - // TODO: Refactor internal symmetric encryption algorithm to be customizable PGPEncryptedDataGenerator encryptedDataGenerator = new PGPEncryptedDataGenerator( - new JcePGPDataEncryptorBuilder(PGPEncryptedData.AES_128).setWithIntegrityPacket(true).setSecureRandom(new SecureRandom()).setProvider(provider)); + new JcePGPDataEncryptorBuilder(cipher).setWithIntegrityPacket(true).setSecureRandom(new SecureRandom()).setProvider(provider)); encryptedDataGenerator.addMethod(new JcePublicKeyKeyEncryptionMethodGenerator(publicKey).setProvider(provider)); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptor.java index 6d5bb6d3c2ef..1e843a7790a4 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptor.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptor.java @@ -25,7 +25,6 @@ import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processors.standard.EncryptContent.Encryptor; import org.bouncycastle.openpgp.PGPCompressedData; -import org.bouncycastle.openpgp.PGPEncryptedData; import org.bouncycastle.openpgp.PGPEncryptedDataList; import org.bouncycastle.openpgp.PGPException; import org.bouncycastle.openpgp.PGPLiteralData; @@ -47,17 +46,19 @@ public class OpenPGPPasswordBasedEncryptor implements Encryptor { private String provider; private char[] password; private String filename; + private Integer cipher; - public OpenPGPPasswordBasedEncryptor(final String algorithm, final String provider, final char[] passphrase, final String filename) { + public OpenPGPPasswordBasedEncryptor(final String algorithm, final Integer cipher, final String provider, final char[] passphrase, final String filename) { this.algorithm = algorithm; this.provider = provider; this.password = passphrase; this.filename = filename; + this.cipher = cipher; } @Override public StreamCallback getEncryptionCallback() throws Exception { - return new OpenPGPEncryptCallback(algorithm, provider, password, filename); + return new OpenPGPEncryptCallback(algorithm, cipher, provider, password, filename); } @Override @@ -136,19 +137,21 @@ private static class OpenPGPEncryptCallback implements StreamCallback { private String provider; private char[] password; private String filename; + private Integer cipher; - OpenPGPEncryptCallback(final String algorithm, final String provider, final char[] password, final String filename) { + OpenPGPEncryptCallback(final String algorithm, final Integer cipher, final String provider, final char[] password, final String filename) { this.algorithm = algorithm; this.provider = provider; this.password = password; this.filename = filename; + this.cipher = cipher; } @Override public void process(InputStream in, OutputStream out) throws IOException { try { PGPKeyEncryptionMethodGenerator encryptionMethodGenerator = new JcePBEKeyEncryptionMethodGenerator(password).setProvider(provider); - org.apache.nifi.processors.standard.util.PGPUtil.encrypt(in, out, algorithm, provider, PGPEncryptedData.AES_128, filename, encryptionMethodGenerator); + org.apache.nifi.processors.standard.util.PGPUtil.encrypt(in, out, algorithm, provider, cipher, filename, encryptionMethodGenerator); } catch (Exception e) { throw new ProcessException(e.getMessage()); } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java index 063652b8c3eb..6464665bb3c0 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java @@ -16,12 +16,20 @@ */ package org.apache.nifi.processors.standard; +import java.io.ByteArrayInputStream; import java.io.File; +import java.io.InputStream; import java.io.IOException; +import java.lang.reflect.Method; import java.nio.file.Paths; import java.security.Security; +import java.util.ArrayList; import java.util.Collection; +import java.util.List; +import java.util.Objects; + import org.apache.commons.codec.binary.Hex; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.security.util.EncryptionMethod; import org.apache.nifi.security.util.KeyDerivationFunction; @@ -31,6 +39,8 @@ import org.apache.nifi.util.MockProcessContext; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; +import org.bouncycastle.bcpg.BCPGInputStream; +import org.bouncycastle.bcpg.SymmetricKeyEncSessionPacket; import org.bouncycastle.jce.provider.BouncyCastleProvider; import org.junit.Assert; import org.junit.Assume; @@ -39,10 +49,25 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.bouncycastle.openpgp.PGPUtil.getDecoderStream; +import static org.junit.Assert.fail; + public class TestEncryptContent { private static final Logger logger = LoggerFactory.getLogger(TestEncryptContent.class); + private static AllowableValue[] getPGPCipherList() { + try{ + Method method = EncryptContent.class.getDeclaredMethod("buildPGPSymmetricCipherAllowableValues"); + method.setAccessible(true); + return ((AllowableValue[]) method.invoke(null)); + } catch (Exception e){ + logger.error("Cannot access buildPGPSymmetricCipherAllowableValues", e); + fail("Cannot access buildPGPSymmetricCipherAllowableValues"); + } + return null; + } + @Before public void setUp() { Security.addProvider(new BouncyCastleProvider()); @@ -92,6 +117,83 @@ public void testRoundTrip() throws IOException { } } + @Test + public void testPGPCiphersRoundTrip() { + final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent()); + testRunner.setProperty(EncryptContent.PASSWORD, "passwordpassword"); // a >=16 characters password + testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name()); + + List pgpAlgorithms = new ArrayList<>(); + pgpAlgorithms.add("PGP"); + pgpAlgorithms.add("PGP_ASCII_ARMOR"); + + for (String algorithm : pgpAlgorithms) { + testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, algorithm); + for (AllowableValue cipher : Objects.requireNonNull(getPGPCipherList())) { + testRunner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, cipher.getValue()); + testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE); + + testRunner.enqueue("A cool plaintext!"); + testRunner.clearTransferState(); + testRunner.run(); + + testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1); + + MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0); + testRunner.assertQueueEmpty(); + + testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE); + // Encryption cipher is inferred from ciphertext, this property deliberately set a fixed cipher to prove + // the output will still be correct + testRunner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, "1"); + + testRunner.enqueue(flowFile); + testRunner.clearTransferState(); + testRunner.run(); + testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1); + + flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0); + flowFile.assertContentEquals("A cool plaintext!"); + } + } + } + + @Test + public void testPGPCiphers() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent()); + testRunner.setProperty(EncryptContent.PASSWORD, "passwordpassword"); // a >= 16 characters password + testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name()); + + List pgpAlgorithms = new ArrayList<>(); + pgpAlgorithms.add("PGP"); + pgpAlgorithms.add("PGP_ASCII_ARMOR"); + + for (String algorithm : pgpAlgorithms) { + + testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, algorithm); + for (AllowableValue cipher : Objects.requireNonNull(getPGPCipherList())) { + testRunner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, cipher.getValue()); + testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE); + + testRunner.enqueue("A cool plaintext!"); + testRunner.clearTransferState(); + testRunner.run(); + + testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1); + + MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0); + testRunner.assertQueueEmpty(); + + // Other than the round trip, checks that the provided cipher is actually used, inferring it from the ciphertext + InputStream ciphertext = new ByteArrayInputStream(flowFile.toByteArray()); + BCPGInputStream pgpin = new BCPGInputStream(getDecoderStream(ciphertext)); + assert pgpin.nextPacketTag() == 3; + assert ((SymmetricKeyEncSessionPacket) pgpin.readPacket()).getEncAlgorithm() == Integer.valueOf(cipher.getValue()); + pgpin.close(); + } + } + } + @Test public void testShouldDetermineMaxKeySizeForAlgorithms() throws IOException { // Arrange @@ -416,5 +518,34 @@ public void testValidation() { " could not be opened with the provided " + EncryptContent.PRIVATE_KEYRING_PASSPHRASE.getDisplayName())); } + runner.removeProperty(EncryptContent.PRIVATE_KEYRING_PASSPHRASE); + + // This configuration is invalid because PGP_SYMMETRIC_ENCRYPTION_CIPHER is outside the allowed [1-13] interval + runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE); + runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, "PGP"); + runner.setProperty(EncryptContent.PASSWORD, "PASSWORD"); + runner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, "256"); + runner.assertNotValid(); + + // This configuration is invalid because PGP_SYMMETRIC_ENCRYPTION_CIPHER points to SAFER cipher which is unsupported + runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE); + runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, "PGP"); + runner.setProperty(EncryptContent.PASSWORD, "PASSWORD"); + runner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, "5"); + runner.assertNotValid(); + + // This configuration is valid + runner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE); + runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, "PGP"); + runner.setProperty(EncryptContent.PASSWORD, "PASSWORD"); + runner.removeProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER); + runner.assertValid(); + + // This configuration is valid because the default value will be used for PGP_SYMMETRIC_ENCRYPTION_CIPHER + runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE); + runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, "PGP"); + runner.setProperty(EncryptContent.PASSWORD, "PASSWORD"); + runner.removeProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER); + runner.assertValid(); } } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java index c823e616d4c6..dd10550517de 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java @@ -29,6 +29,8 @@ import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.security.util.EncryptionMethod; import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.bouncycastle.openpgp.PGPEncryptedData; +import org.bouncycastle.openpgp.PGPUtil; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -67,65 +69,77 @@ public void tearDown() throws Exception { @Test public void testShouldEncryptAndDecrypt() throws Exception { - // Arrange - final String PLAINTEXT = "This is a plaintext message."; - logger.info("Plaintext: {}", PLAINTEXT); - InputStream plainStream = new ByteArrayInputStream(PLAINTEXT.getBytes("UTF-8")); - OutputStream cipherStream = new ByteArrayOutputStream(); - OutputStream recoveredStream = new ByteArrayOutputStream(); - - // No file, just streams - String filename = "tempFile.txt"; - - // Encryptor does not require password - OpenPGPKeyBasedEncryptor encryptor = new OpenPGPKeyBasedEncryptor( - EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), PUBLIC_KEYRING_PATH, USER_ID, new char[0], filename); - StreamCallback encryptionCallback = encryptor.getEncryptionCallback(); - - OpenPGPKeyBasedEncryptor decryptor = new OpenPGPKeyBasedEncryptor( - EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), SECRET_KEYRING_PATH, USER_ID, PASSWORD.toCharArray(), filename); - StreamCallback decryptionCallback = decryptor.getDecryptionCallback(); - - // Act - encryptionCallback.process(plainStream, cipherStream); - - final byte[] cipherBytes = ((ByteArrayOutputStream) cipherStream).toByteArray(); - logger.info("Encrypted: {}", Hex.encodeHexString(cipherBytes)); - InputStream cipherInputStream = new ByteArrayInputStream(cipherBytes); - - decryptionCallback.process(cipherInputStream, recoveredStream); - - // Assert - byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray(); - String recovered = new String(recoveredBytes, "UTF-8"); - logger.info("Recovered: {}", recovered); - assert PLAINTEXT.equals(recovered); + for (int i = 1; i < 14; i++) { + if (PGPEncryptedData.SAFER != i) { // SAFER cipher is not supported and therefore its test is skipped + Integer cipher = i; + logger.info("Testing PGP encryption with " + PGPUtil.getSymmetricCipherName(cipher) + " cipher."); + // Arrange + final String PLAINTEXT = "This is a plaintext message."; + logger.info("Plaintext: {}", PLAINTEXT); + InputStream plainStream = new ByteArrayInputStream(PLAINTEXT.getBytes("UTF-8")); + OutputStream cipherStream = new ByteArrayOutputStream(); + OutputStream recoveredStream = new ByteArrayOutputStream(); + + // No file, just streams + String filename = "tempFile.txt"; + + + // Encryptor does not require password + OpenPGPKeyBasedEncryptor encryptor = new OpenPGPKeyBasedEncryptor( + EncryptionMethod.PGP.getAlgorithm(), cipher, EncryptionMethod.PGP.getProvider(), PUBLIC_KEYRING_PATH, USER_ID, new char[0], filename); + StreamCallback encryptionCallback = encryptor.getEncryptionCallback(); + + OpenPGPKeyBasedEncryptor decryptor = new OpenPGPKeyBasedEncryptor( + EncryptionMethod.PGP.getAlgorithm(), cipher, EncryptionMethod.PGP.getProvider(), SECRET_KEYRING_PATH, USER_ID, PASSWORD.toCharArray(), filename); + StreamCallback decryptionCallback = decryptor.getDecryptionCallback(); + + // Act + encryptionCallback.process(plainStream, cipherStream); + + final byte[] cipherBytes = ((ByteArrayOutputStream) cipherStream).toByteArray(); + logger.info("Encrypted: {}", Hex.encodeHexString(cipherBytes)); + InputStream cipherInputStream = new ByteArrayInputStream(cipherBytes); + + decryptionCallback.process(cipherInputStream, recoveredStream); + + // Assert + byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray(); + String recovered = new String(recoveredBytes, "UTF-8"); + logger.info("Recovered: {}", recovered); + assert PLAINTEXT.equals(recovered); + } + } } @Test public void testShouldDecryptExternalFile() throws Exception { - // Arrange - byte[] plainBytes = Files.readAllBytes(Paths.get(plainFile.getPath())); - final String PLAINTEXT = new String(plainBytes, "UTF-8"); - - InputStream cipherStream = new FileInputStream(unsignedFile); - OutputStream recoveredStream = new ByteArrayOutputStream(); - - // No file, just streams - String filename = unsignedFile.getName(); - - OpenPGPKeyBasedEncryptor encryptor = new OpenPGPKeyBasedEncryptor( - EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), SECRET_KEYRING_PATH, USER_ID, PASSWORD.toCharArray(), filename); - - StreamCallback decryptionCallback = encryptor.getDecryptionCallback(); - - // Act - decryptionCallback.process(cipherStream, recoveredStream); - - // Assert - byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray(); - String recovered = new String(recoveredBytes, "UTF-8"); - logger.info("Recovered: {}", recovered); - Assert.assertEquals("Recovered text", PLAINTEXT, recovered); + for (int i = 1; i<14; i++) { + if (PGPEncryptedData.SAFER != i) { // SAFER cipher is not supported and therefore its test is skipped + Integer cipher = i; + // Arrange + byte[] plainBytes = Files.readAllBytes(Paths.get(plainFile.getPath())); + final String PLAINTEXT = new String(plainBytes, "UTF-8"); + + InputStream cipherStream = new FileInputStream(unsignedFile); + OutputStream recoveredStream = new ByteArrayOutputStream(); + + // No file, just streams + String filename = unsignedFile.getName(); + + OpenPGPKeyBasedEncryptor encryptor = new OpenPGPKeyBasedEncryptor( + EncryptionMethod.PGP.getAlgorithm(), cipher, EncryptionMethod.PGP.getProvider(), SECRET_KEYRING_PATH, USER_ID, PASSWORD.toCharArray(), filename); + + StreamCallback decryptionCallback = encryptor.getDecryptionCallback(); + + // Act + decryptionCallback.process(cipherStream, recoveredStream); + + // Assert + byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray(); + String recovered = new String(recoveredBytes, "UTF-8"); + logger.info("Recovered: {}", recovered); + Assert.assertEquals("Recovered text", PLAINTEXT, recovered); + } + } } -} \ No newline at end of file +} diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java index 2e1cd5f36712..3ad926bedd76 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java @@ -29,6 +29,8 @@ import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.security.util.EncryptionMethod; import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.bouncycastle.openpgp.PGPEncryptedData; +import org.bouncycastle.openpgp.PGPUtil; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -63,60 +65,75 @@ public void tearDown() throws Exception { @Test public void testShouldEncryptAndDecrypt() throws Exception { - // Arrange - final String PLAINTEXT = "This is a plaintext message."; - logger.info("Plaintext: {}", PLAINTEXT); - InputStream plainStream = new java.io.ByteArrayInputStream(PLAINTEXT.getBytes("UTF-8")); - OutputStream cipherStream = new ByteArrayOutputStream(); - OutputStream recoveredStream = new ByteArrayOutputStream(); - // No file, just streams - String filename = "tempFile.txt"; + for (int i = 1; i<14; i++) { + if (PGPEncryptedData.SAFER != i) { // SAFER cipher is not supported and therefore its test is skipped + Integer cipher = i; + logger.info("Testing PGP encryption with " + PGPUtil.getSymmetricCipherName(cipher) + " cipher."); - OpenPGPPasswordBasedEncryptor encryptor = new OpenPGPPasswordBasedEncryptor(EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), PASSWORD.toCharArray(), filename); + // Arrange + final String PLAINTEXT = "This is a plaintext message."; + logger.info("Plaintext: {}", PLAINTEXT); + InputStream plainStream = new java.io.ByteArrayInputStream(PLAINTEXT.getBytes("UTF-8")); + OutputStream cipherStream = new ByteArrayOutputStream(); + OutputStream recoveredStream = new ByteArrayOutputStream(); - StreamCallback encryptionCallback = encryptor.getEncryptionCallback(); - StreamCallback decryptionCallback = encryptor.getDecryptionCallback(); + // No file, just streams + String filename = "tempFile.txt"; - // Act - encryptionCallback.process(plainStream, cipherStream); + OpenPGPPasswordBasedEncryptor encryptor = new OpenPGPPasswordBasedEncryptor(EncryptionMethod.PGP.getAlgorithm(), + cipher, EncryptionMethod.PGP.getProvider(), PASSWORD.toCharArray(), filename); - final byte[] cipherBytes = ((ByteArrayOutputStream) cipherStream).toByteArray(); - logger.info("Encrypted: {}", Hex.encodeHexString(cipherBytes)); - InputStream cipherInputStream = new ByteArrayInputStream(cipherBytes); + StreamCallback encryptionCallback = encryptor.getEncryptionCallback(); + StreamCallback decryptionCallback = encryptor.getDecryptionCallback(); - decryptionCallback.process(cipherInputStream, recoveredStream); + // Act + encryptionCallback.process(plainStream, cipherStream); - // Assert - byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray(); - String recovered = new String(recoveredBytes, "UTF-8"); - logger.info("Recovered: {}", recovered); - assert PLAINTEXT.equals(recovered); + final byte[] cipherBytes = ((ByteArrayOutputStream) cipherStream).toByteArray(); + logger.info("Encrypted: {}", Hex.encodeHexString(cipherBytes)); + InputStream cipherInputStream = new ByteArrayInputStream(cipherBytes); + + decryptionCallback.process(cipherInputStream, recoveredStream); + + // Assert + byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray(); + String recovered = new String(recoveredBytes, "UTF-8"); + logger.info("Recovered: {}", recovered); + assert PLAINTEXT.equals(recovered); + } + } } @Test public void testShouldDecryptExternalFile() throws Exception { - // Arrange - byte[] plainBytes = Files.readAllBytes(Paths.get(plainFile.getPath())); - final String PLAINTEXT = new String(plainBytes, "UTF-8"); - - InputStream cipherStream = new FileInputStream(encryptedFile); - OutputStream recoveredStream = new ByteArrayOutputStream(); - - // No file, just streams - String filename = encryptedFile.getName(); - - OpenPGPPasswordBasedEncryptor encryptor = new OpenPGPPasswordBasedEncryptor(EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), LEGACY_PASSWORD.toCharArray(), filename); - - StreamCallback decryptionCallback = encryptor.getDecryptionCallback(); - - // Act - decryptionCallback.process(cipherStream, recoveredStream); - - // Assert - byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray(); - String recovered = new String(recoveredBytes, "UTF-8"); - logger.info("Recovered: {}", recovered); - Assert.assertEquals("Recovered text", PLAINTEXT, recovered); + for (int i = 1; i<14; i++) { + if (PGPEncryptedData.SAFER != i) { // SAFER cipher is not supported and therefore its test is skipped + Integer cipher = i; + // Arrange + byte[] plainBytes = Files.readAllBytes(Paths.get(plainFile.getPath())); + final String PLAINTEXT = new String(plainBytes, "UTF-8"); + + InputStream cipherStream = new FileInputStream(encryptedFile); + OutputStream recoveredStream = new ByteArrayOutputStream(); + + // No file, just streams + String filename = encryptedFile.getName(); + + OpenPGPPasswordBasedEncryptor encryptor = new OpenPGPPasswordBasedEncryptor(EncryptionMethod.PGP.getAlgorithm(), cipher, + EncryptionMethod.PGP.getProvider(), LEGACY_PASSWORD.toCharArray(), filename); + + StreamCallback decryptionCallback = encryptor.getDecryptionCallback(); + + // Act + decryptionCallback.process(cipherStream, recoveredStream); + + // Assert + byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray(); + String recovered = new String(recoveredBytes, "UTF-8"); + logger.info("Recovered: {}", recovered); + Assert.assertEquals("Recovered text", PLAINTEXT, recovered); + } + } } -} \ No newline at end of file +} From 768a7b8c004f2ab0a4284ab29479cd917f486c1b Mon Sep 17 00:00:00 2001 From: Mike Thomsen Date: Thu, 4 Jul 2019 07:50:21 -0400 Subject: [PATCH 07/11] NIFI-6425 Made executeQuery able to reconnect to the Gremlin cluster. Signed-off-by: Matthew Burgess This closes #3572 --- .../nifi/graph/GremlinClientService.java | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java index a79094f42706..e001fd0493bd 100644 --- a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java +++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java @@ -37,9 +37,11 @@ public class GremlinClientService extends AbstractTinkerpopClientService impleme private Cluster cluster; protected Client client; public static final String NOT_SUPPORTED = "NOT_SUPPORTED"; + private ConfigurationContext context; @OnEnabled public void onEnabled(ConfigurationContext context) { + this.context = context; cluster = buildCluster(context); client = cluster.connect(); } @@ -52,8 +54,7 @@ public void onDisabled() { cluster = null; } - @Override - public Map executeQuery(String query, Map parameters, GraphQueryResultCallback handler) { + public Map doQuery(String query, Map parameters, GraphQueryResultCallback handler) { try { Iterator iterator = client.submit(query, parameters).iterator(); long count = 0; @@ -86,6 +87,19 @@ public Map executeQuery(String query, Map parame } } + @Override + public Map executeQuery(String query, Map parameters, GraphQueryResultCallback handler) { + try { + return doQuery(query, parameters, handler); + } catch (Exception ex) { + cluster.close(); + client.close(); + cluster = buildCluster(context); + client = cluster.connect(); + return doQuery(query, parameters, handler); + } + } + @Override public String getTransitUrl() { return transitUrl; From 9071e5baa7a8af03392fce703d539e9fa94980be Mon Sep 17 00:00:00 2001 From: Alessandro D'Armiento Date: Sat, 3 Aug 2019 13:49:44 +0200 Subject: [PATCH 08/11] - Removed unused AUTOCREATE_PARTITIONS from PutHive3Streaming - Renamed PARTITION_VALUES to STATIC_PARTITION_VALUES for correctness and better understanding - STATIC_PARTITION_VALUES descriptions clearly states that having that property filler implies Hive Static Partitioning NIFI-6536: Additional documentation for Static Partition Values Signed-off-by: Matthew Burgess This closes #3631 --- .../processors/hive/PutHive3Streaming.java | 35 +++++++------------ .../apache/nifi/util/hive/HiveOptions.java | 6 ---- 2 files changed, 12 insertions(+), 29 deletions(-) diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java index 2224b0649340..5558c79eb448 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java @@ -84,9 +84,10 @@ import static org.apache.nifi.processors.hive.AbstractHive3QLProcessor.ATTR_OUTPUT_TABLES; @Tags({"hive", "streaming", "put", "database", "store"}) -@CapabilityDescription("This processor uses Hive Streaming to send flow file records to an Apache Hive 3.0+ table. " - + "The partition values are expected to be the 'last' fields of each record, so if the table is partitioned on column A for example, then the last field in " - + "each record should be field A.") +@CapabilityDescription("This processor uses Hive Streaming to send flow file records to an Apache Hive 3.0+ table. If 'Static Partition Values' is not set, then " + + "the partition values are expected to be the 'last' fields of each record, so if the table is partitioned on column A for example, then the last field in " + + "each record should be field A. If 'Static Partition Values' is set, those values will be used as the partition values, and any record fields corresponding to " + + "partition columns will be ignored.") @WritesAttributes({ @WritesAttribute(attribute = "hivestreaming.record.count", description = "This attribute is written on the flow files routed to the 'success' " + "and 'failure' relationships, and contains the number of records from the incoming flow file. All records in a flow file are committed as a single transaction."), @@ -151,28 +152,19 @@ public class PutHive3Streaming extends AbstractProcessor { .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .build(); - static final PropertyDescriptor PARTITION_VALUES = new PropertyDescriptor.Builder() + static final PropertyDescriptor STATIC_PARTITION_VALUES = new PropertyDescriptor.Builder() .name("hive3-stream-part-vals") - .displayName("Partition Values") + .displayName("Static Partition Values") .description("Specifies a comma-separated list of the values for the partition columns of the target table. If the incoming records all have the same values " + "for the partition columns, those values can be entered here, resulting in a performance gain. If specified, this property will often contain " + "Expression Language, for example if PartitionRecord is upstream and two partitions 'name' and 'age' are used, then this property can be set to " - + "${name},${age}.") + + "${name},${age}. If this property is set, the values will be used as the partition values, and any record fields corresponding to " + + "partition columns will be ignored. If this property is not set, then the partition values are expected to be the last fields of each record.") .required(false) .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .build(); - static final PropertyDescriptor AUTOCREATE_PARTITIONS = new PropertyDescriptor.Builder() - .name("hive3-stream-autocreate-partition") - .displayName("Auto-Create Partitions") - .description("Flag indicating whether partitions should be automatically created") - .required(true) - .addValidator(StandardValidators.BOOLEAN_VALIDATOR) - .allowableValues("true", "false") - .defaultValue("true") - .build(); - static final PropertyDescriptor CALL_TIMEOUT = new PropertyDescriptor.Builder() .name("hive3-stream-call-timeout") .displayName("Call Timeout") @@ -250,8 +242,7 @@ protected void init(ProcessorInitializationContext context) { props.add(HIVE_CONFIGURATION_RESOURCES); props.add(DB_NAME); props.add(TABLE_NAME); - props.add(PARTITION_VALUES); - props.add(AUTOCREATE_PARTITIONS); + props.add(STATIC_PARTITION_VALUES); props.add(CALL_TIMEOUT); props.add(DISABLE_STREAMING_OPTIMIZATIONS); props.add(ROLLBACK_ON_FAILURE); @@ -362,8 +353,7 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro } } - final String partitionValuesString = context.getProperty(PARTITION_VALUES).evaluateAttributeExpressions(flowFile).getValue(); - final boolean autoCreatePartitions = context.getProperty(AUTOCREATE_PARTITIONS).asBoolean(); + final String staticPartitionValuesString = context.getProperty(STATIC_PARTITION_VALUES).evaluateAttributeExpressions(flowFile).getValue(); final boolean disableStreamingOptimizations = context.getProperty(DISABLE_STREAMING_OPTIMIZATIONS).asBoolean(); // Override the Hive Metastore URIs in the config if set by the user @@ -373,12 +363,11 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro HiveOptions o = new HiveOptions(metastoreURIs, dbName, tableName) .withHiveConf(hiveConfig) - .withAutoCreatePartitions(autoCreatePartitions) .withCallTimeout(callTimeout) .withStreamingOptimizations(!disableStreamingOptimizations); - if (!StringUtils.isEmpty(partitionValuesString)) { - List staticPartitionValues = Arrays.stream(partitionValuesString.split(",")).filter(Objects::nonNull).map(String::trim).collect(Collectors.toList()); + if (!StringUtils.isEmpty(staticPartitionValuesString)) { + List staticPartitionValues = Arrays.stream(staticPartitionValuesString.split(",")).filter(Objects::nonNull).map(String::trim).collect(Collectors.toList()); o = o.withStaticPartitionValues(staticPartitionValues); } diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java index ca6e6ebbe95e..82f6856160ab 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java @@ -32,7 +32,6 @@ public class HiveOptions implements Serializable { protected Integer idleTimeout = 60000; protected Integer callTimeout = 0; protected List staticPartitionValues = null; - protected Boolean autoCreatePartitions = true; protected String kerberosPrincipal; protected String kerberosKeytab; protected HiveConf hiveConf; @@ -54,11 +53,6 @@ public HiveOptions withStaticPartitionValues(List staticPartitionValues) return this; } - public HiveOptions withAutoCreatePartitions(Boolean autoCreatePartitions) { - this.autoCreatePartitions = autoCreatePartitions; - return this; - } - public HiveOptions withKerberosKeytab(String kerberosKeytab) { this.kerberosKeytab = kerberosKeytab; return this; From 76c2c3fee298ae33775ce794fef7f3a0851eb179 Mon Sep 17 00:00:00 2001 From: Bryan Bende Date: Fri, 23 Aug 2019 16:15:34 -0400 Subject: [PATCH 09/11] NIFI-6089 Add Parquet record reader and writer NIFI-5755 Allow PutParquet processor to specify avro write configuration Review feedback Additional review feedback This closes #3679 Signed-off-by: Mike Thomsen --- .../azure/eventhub/ConsumeAzureEventHub.java | 8 +- .../eventhub/TestConsumeAzureEventHub.java | 3 +- .../CouchbaseRecordLookupService.java | 2 +- .../nifi-avro-record-utils/pom.xml | 4 + .../SchemaRegistryRecordSetWriter.java | 0 .../record/ArrayListRecordReader.java | 2 +- .../record/CommaSeparatedRecordReader.java | 18 +- .../record/MockRecordParser.java | 18 +- .../listen/SSLSocketChannelRecordReader.java | 6 +- .../listen/SocketChannelRecordReader.java | 9 +- .../StandardSocketChannelRecordReader.java | 6 +- .../CacheIdSchemaAccessWriter.java | 0 .../hive/TestPutHive3Streaming.java | 4 +- .../kafka/pubsub/ConsumerLease.java | 47 +++-- .../kafka/pubsub/PublishKafkaRecord_0_10.java | 2 +- .../kafka/pubsub/TestPublisherLease.java | 34 ++-- .../kafka/pubsub/util/MockRecordParser.java | 20 +- .../kafka/pubsub/ConsumerLease.java | 2 +- .../kafka/pubsub/TestPublisherLease.java | 32 +-- .../kafka/pubsub/util/MockRecordParser.java | 20 +- .../kafka/pubsub/ConsumerLease.java | 2 +- .../kafka/pubsub/TestPublisherLease.java | 32 +-- .../kafka/pubsub/util/MockRecordParser.java | 20 +- .../kafka/pubsub/ConsumerLease.java | 2 +- .../kafka/pubsub/TestPublisherLease.java | 4 +- .../kafka/pubsub/util/MockRecordParser.java | 2 +- .../apache/nifi/parquet/ParquetReader.java | 59 ++++++ .../nifi/parquet/ParquetRecordSetWriter.java | 116 +++++++++++ .../hadoop}/AvroParquetHDFSRecordReader.java | 2 +- .../hadoop}/AvroParquetHDFSRecordWriter.java | 2 +- .../parquet/record/ParquetRecordReader.java | 98 ++++++++++ .../parquet/record/WriteParquetResult.java | 81 ++++++++ .../parquet/stream/NifiOutputStream.java | 2 +- .../parquet/stream/NifiParquetInputFile.java | 53 +++++ .../parquet/stream/NifiParquetOutputFile.java | 2 +- .../stream/NifiSeekableInputStream.java | 71 +++++++ .../nifi/parquet/utils/ParquetConfig.java | 133 +++++++++++++ .../parquet/utils/ParquetUtils.java | 182 ++++++++++++++---- .../parquet/ConvertAvroToParquet.java | 33 ++-- .../nifi/processors/parquet/FetchParquet.java | 2 +- .../nifi/processors/parquet/PutParquet.java | 25 ++- ...g.apache.nifi.controller.ControllerService | 17 ++ .../nifi/parquet/TestParquetReader.java | 119 ++++++++++++ .../parquet/TestParquetRecordSetWriter.java | 159 +++++++++++++++ .../processors/parquet/PutParquetTest.java | 2 +- .../nifi/record/script/ScriptedReader.java | 4 +- .../record/script/ScriptedReaderTest.groovy | 13 +- .../groovy/test_record_reader_inline.groovy | 2 +- .../groovy/test_record_reader_xml.groovy | 2 +- .../standard/AbstractRecordProcessor.java | 2 +- .../standard/AbstractRouteRecord.java | 2 +- .../nifi/processors/standard/ForkRecord.java | 30 +-- .../processors/standard/ListenTCPRecord.java | 4 +- .../processors/standard/ListenUDPRecord.java | 3 +- .../processors/standard/PartitionRecord.java | 34 ++-- .../nifi/processors/standard/QueryRecord.java | 2 +- .../nifi/processors/standard/SplitRecord.java | 24 +-- .../processors/standard/TestForkRecord.java | 21 +- .../standard/TestValidateRecord.java | 21 +- .../apache/nifi/lookup/RestLookupService.java | 6 +- .../serialization/RecordReaderFactory.java | 7 +- .../java/org/apache/nifi/avro/AvroReader.java | 2 +- .../java/org/apache/nifi/csv/CSVReader.java | 2 +- .../java/org/apache/nifi/grok/GrokReader.java | 2 +- .../org/apache/nifi/json/JsonPathReader.java | 3 +- .../org/apache/nifi/json/JsonTreeReader.java | 3 +- .../apache/nifi/syslog/Syslog5424Reader.java | 2 +- .../org/apache/nifi/syslog/SyslogReader.java | 2 +- .../java/org/apache/nifi/xml/XMLReader.java | 3 +- 69 files changed, 1339 insertions(+), 314 deletions(-) rename nifi-nar-bundles/{nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services => nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils}/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java (100%) rename nifi-nar-bundles/{nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services => nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils}/src/main/java/org/apache/nifi/serialization/CacheIdSchemaAccessWriter.java (100%) create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/ParquetReader.java create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/ParquetRecordSetWriter.java rename nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/{processors/parquet/record => parquet/hadoop}/AvroParquetHDFSRecordReader.java (97%) rename nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/{processors/parquet/record => parquet/hadoop}/AvroParquetHDFSRecordWriter.java (97%) create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/record/ParquetRecordReader.java create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/record/WriteParquetResult.java rename nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/{processors => }/parquet/stream/NifiOutputStream.java (97%) create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiParquetInputFile.java rename nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/{processors => }/parquet/stream/NifiParquetOutputFile.java (96%) create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiSeekableInputStream.java create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/utils/ParquetConfig.java rename nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/{processors => }/parquet/utils/ParquetUtils.java (55%) create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetReader.java create mode 100644 nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetRecordSetWriter.java diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java index fb7699f8c105..e9bafb0cfa4f 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java @@ -48,6 +48,7 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.RecordReader; import org.apache.nifi.serialization.RecordReaderFactory; @@ -74,7 +75,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; -import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils; import static org.apache.nifi.util.StringUtils.isEmpty; @@ -454,9 +454,9 @@ private void writeRecords(PartitionContext context, Iterable messages try (final OutputStream out = session.write(flowFile)) { for (final EventData eventData : messages) { - - try (final InputStream in = new ByteArrayInputStream(eventData.getBytes())) { - final RecordReader reader = readerFactory.createRecordReader(schemaRetrievalVariables, in, logger); + final byte[] eventDataBytes = eventData.getBytes(); + try (final InputStream in = new ByteArrayInputStream(eventDataBytes)) { + final RecordReader reader = readerFactory.createRecordReader(schemaRetrievalVariables, in, eventDataBytes.length, logger); Record record; while ((record = reader.nextRecord()) != null) { diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/TestConsumeAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/TestConsumeAzureEventHub.java index 3946cba0bf46..6f350937bd29 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/TestConsumeAzureEventHub.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/TestConsumeAzureEventHub.java @@ -59,6 +59,7 @@ import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -204,7 +205,7 @@ private void setupRecordReader(List eventDataList, int throwException final RecordReaderFactory readerFactory = mock(RecordReaderFactory.class); processor.setReaderFactory(readerFactory); final RecordReader reader = mock(RecordReader.class); - when(readerFactory.createRecordReader(anyMap(), any(), any())).thenReturn(reader); + when(readerFactory.createRecordReader(anyMap(), any(), anyLong(), any())).thenReturn(reader); final List recordList = eventDataList.stream() .map(eventData -> toRecord(new String(eventData.getBytes()))) .collect(Collectors.toList()); diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/couchbase/CouchbaseRecordLookupService.java b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/couchbase/CouchbaseRecordLookupService.java index e12eac9959e5..32179c61399f 100644 --- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/couchbase/CouchbaseRecordLookupService.java +++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/src/main/java/org/apache/nifi/couchbase/CouchbaseRecordLookupService.java @@ -114,7 +114,7 @@ public Optional lookup(Map coordinates) throws LookupFai recordReaderVariables.put(k, value.toString()); } }); - return new Tuple<>(null, readerFactory.createRecordReader(recordReaderVariables, in, getLogger())); + return new Tuple<>(null, readerFactory.createRecordReader(recordReaderVariables, in, -1, getLogger())); } catch (Exception e) { return new Tuple<>(e, null); } diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/pom.xml index 7884aae30b06..1a5c8dc02fab 100755 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/pom.xml +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/pom.xml @@ -49,6 +49,10 @@ org.apache.nifi nifi-schema-registry-service-api + + org.apache.nifi + nifi-record-serialization-service-api + org.apache.nifi nifi-record diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java similarity index 100% rename from nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java rename to nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/ArrayListRecordReader.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/ArrayListRecordReader.java index 5586ede47756..ea6c02d61e41 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/ArrayListRecordReader.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/ArrayListRecordReader.java @@ -36,7 +36,7 @@ public ArrayListRecordReader(final RecordSchema schema) { } @Override - public ArrayListReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) { + public ArrayListReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) { return new ArrayListReader(records, schema); } diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/CommaSeparatedRecordReader.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/CommaSeparatedRecordReader.java index 6597b75e4de5..560955ccce06 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/CommaSeparatedRecordReader.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/CommaSeparatedRecordReader.java @@ -17,6 +17,14 @@ package org.apache.nifi.serialization.record; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.SimpleRecordSchema; + import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; @@ -26,14 +34,6 @@ import java.util.List; import java.util.Map; -import org.apache.nifi.controller.AbstractControllerService; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.schema.access.SchemaNotFoundException; -import org.apache.nifi.serialization.MalformedRecordException; -import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RecordReaderFactory; -import org.apache.nifi.serialization.SimpleRecordSchema; - public class CommaSeparatedRecordReader extends AbstractControllerService implements RecordReaderFactory { private int failAfterN; private int recordCount = 0; @@ -51,7 +51,7 @@ public void failAfter(final int failAfterN) { } @Override - public RecordReader createRecordReader(Map variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(Map variables, final InputStream in, final long inputLength, final ComponentLog logger) throws IOException, SchemaNotFoundException { final BufferedReader reader = new BufferedReader(new InputStreamReader(in)); final List fields = new ArrayList<>(); diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java index be4046cb54ba..9081d008c7e7 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java @@ -17,14 +17,6 @@ package org.apache.nifi.serialization.record; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.schema.access.SchemaNotFoundException; @@ -33,6 +25,14 @@ import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.SimpleRecordSchema; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + public class MockRecordParser extends AbstractControllerService implements RecordReaderFactory { private final List records = new ArrayList<>(); private final List fields = new ArrayList<>(); @@ -67,7 +67,7 @@ public void addRecord(Object... values) { } @Override - public RecordReader createRecordReader(Map variables, InputStream in, ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(Map variables, InputStream in, long inputLength, ComponentLog logger) throws IOException, SchemaNotFoundException { final Iterator itr = records.iterator(); return new RecordReader() { diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SSLSocketChannelRecordReader.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SSLSocketChannelRecordReader.java index 12a3f3e9edec..f393419e33ba 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SSLSocketChannelRecordReader.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SSLSocketChannelRecordReader.java @@ -16,7 +16,6 @@ */ package org.apache.nifi.record.listen; -import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel; import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelInputStream; @@ -30,6 +29,7 @@ import java.io.InputStream; import java.net.InetAddress; import java.nio.channels.SocketChannel; +import java.util.Collections; /** * Encapsulates an SSLSocketChannel and a RecordReader created for the given channel. @@ -54,14 +54,14 @@ public SSLSocketChannelRecordReader(final SocketChannel socketChannel, } @Override - public RecordReader createRecordReader(final FlowFile flowFile, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException { + public RecordReader createRecordReader(final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException { if (recordReader != null) { throw new IllegalStateException("Cannot create RecordReader because already created"); } final InputStream socketIn = new SSLSocketChannelInputStream(sslSocketChannel); final InputStream in = new BufferedInputStream(socketIn); - recordReader = readerFactory.createRecordReader(flowFile, in, logger); + recordReader = readerFactory.createRecordReader(Collections.emptyMap(), in, -1, logger); return recordReader; } diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SocketChannelRecordReader.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SocketChannelRecordReader.java index b648b7753d61..72f931d5cb04 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SocketChannelRecordReader.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SocketChannelRecordReader.java @@ -16,7 +16,6 @@ */ package org.apache.nifi.record.listen; -import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; @@ -32,18 +31,14 @@ public interface SocketChannelRecordReader extends Closeable { /** - * Currently a RecordReader can only be created with a FlowFile. Since we won't have a FlowFile at the time - * a connection is accepted, this method will be used to lazily create the RecordReader later. Eventually this - * method should be removed and the reader should be passed in through the constructor. + * Lazily creates the RecordReader. * - * - * @param flowFile the flow file we are creating the reader for * @param logger the logger of the component creating the reader * @return a RecordReader * * @throws IllegalStateException if create is called after a reader has already been created */ - RecordReader createRecordReader(final FlowFile flowFile, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException; + RecordReader createRecordReader(ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException; /** * @return the RecordReader created by calling createRecordReader, or null if one has not been created yet diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/StandardSocketChannelRecordReader.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/StandardSocketChannelRecordReader.java index 1e220442c04f..a2da2870c8dc 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/StandardSocketChannelRecordReader.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/StandardSocketChannelRecordReader.java @@ -16,7 +16,6 @@ */ package org.apache.nifi.record.listen; -import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; @@ -27,6 +26,7 @@ import java.io.InputStream; import java.net.InetAddress; import java.nio.channels.SocketChannel; +import java.util.Collections; /** * Encapsulates a SocketChannel and a RecordReader created for the given channel. @@ -48,13 +48,13 @@ public StandardSocketChannelRecordReader(final SocketChannel socketChannel, } @Override - public RecordReader createRecordReader(final FlowFile flowFile, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException { + public RecordReader createRecordReader(final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException { if (recordReader != null) { throw new IllegalStateException("Cannot create RecordReader because already created"); } final InputStream in = socketChannel.socket().getInputStream(); - recordReader = readerFactory.createRecordReader(flowFile, in, logger); + recordReader = readerFactory.createRecordReader(Collections.emptyMap(), in, -1, logger); return recordReader; } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/CacheIdSchemaAccessWriter.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/serialization/CacheIdSchemaAccessWriter.java similarity index 100% rename from nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/CacheIdSchemaAccessWriter.java rename to nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/serialization/CacheIdSchemaAccessWriter.java diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java index c82f55c34cb0..4f49932d0f62 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java @@ -171,11 +171,11 @@ private void configure(final PutHive3Streaming processor, final int numUsers, fi runner.setProperty(PutHive3Streaming.HIVE_CONFIGURATION_RESOURCES, TEST_CONF_PATH); MockRecordParser readerFactory = new MockRecordParser() { @Override - public RecordReader createRecordReader(Map variables, InputStream in, ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(Map variables, InputStream in, long inputLength, ComponentLog logger) throws IOException, SchemaNotFoundException { if (failOnCreateReader) { throw new SchemaNotFoundException("test"); } - return super.createRecordReader(variables, in, logger); + return super.createRecordReader(variables, in, inputLength, logger); } }; final RecordSchema recordSchema = AvroTypeUtil.createSchema(schema); diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java index 2eaa58fe173a..d9772692b951 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java @@ -16,29 +16,6 @@ */ package org.apache.nifi.processors.kafka.pubsub; -import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_10.REL_PARSE_FAILURE; -import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_10.REL_SUCCESS; -import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING; -import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING; - -import java.io.ByteArrayInputStream; -import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.TimeUnit; -import java.util.function.BiConsumer; - -import javax.xml.bind.DatatypeConverter; - import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -60,6 +37,28 @@ import org.apache.nifi.serialization.record.Record; import org.apache.nifi.serialization.record.RecordSchema; +import javax.xml.bind.DatatypeConverter; +import java.io.ByteArrayInputStream; +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; + +import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_10.REL_PARSE_FAILURE; +import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_10.REL_SUCCESS; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING; + /** * This class represents a lease to access a Kafka Consumer object. The lease is * intended to be obtained from a ConsumerPool. The lease is closeable to allow @@ -471,7 +470,7 @@ private void writeRecordData(final ProcessSession session, final List records = new ArrayList<>(); private final List fields = new ArrayList<>(); @@ -63,7 +63,7 @@ public void addRecord(Object... values) { } @Override - public RecordReader createRecordReader(Map variables, InputStream in, ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(Map variables, InputStream in, long inputLength, ComponentLog logger) throws IOException, SchemaNotFoundException { final BufferedReader reader = new BufferedReader(new InputStreamReader(in)); return new RecordReader() { diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java index 36e03ea0194f..bcba8ac9fb7f 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java @@ -520,7 +520,7 @@ private void writeRecordData(final ProcessSession session, final List records = new ArrayList<>(); private final List fields = new ArrayList<>(); @@ -63,7 +63,7 @@ public void addRecord(Object... values) { } @Override - public RecordReader createRecordReader(Map variables, InputStream in, ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(Map variables, InputStream in, long inputLength, ComponentLog logger) throws IOException, SchemaNotFoundException { final BufferedReader reader = new BufferedReader(new InputStreamReader(in)); return new RecordReader() { diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java index f2a382dcabbe..04176144f188 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java @@ -520,7 +520,7 @@ private void writeRecordData(final ProcessSession session, final List records = new ArrayList<>(); private final List fields = new ArrayList<>(); @@ -63,7 +63,7 @@ public void addRecord(Object... values) { } @Override - public RecordReader createRecordReader(Map variables, InputStream in, ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(Map variables, InputStream in, long inputLength, ComponentLog logger) throws IOException, SchemaNotFoundException { final BufferedReader reader = new BufferedReader(new InputStreamReader(in)); return new RecordReader() { diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java index 83aa14b95e7c..77d53a979d88 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java @@ -520,7 +520,7 @@ private void writeRecordData(final ProcessSession session, final List variables, InputStream in, ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(Map variables, InputStream in, long inputLength, ComponentLog logger) throws IOException, SchemaNotFoundException { final BufferedReader reader = new BufferedReader(new InputStreamReader(in)); return new RecordReader() { diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/ParquetReader.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/ParquetReader.java new file mode 100644 index 000000000000..94d2101e6754 --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/ParquetReader.java @@ -0,0 +1,59 @@ +/* + * 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.parquet; + +import org.apache.hadoop.conf.Configuration; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.parquet.record.ParquetRecordReader; +import org.apache.nifi.parquet.utils.ParquetConfig; +import org.apache.nifi.parquet.utils.ParquetUtils; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.apache.nifi.parquet.utils.ParquetUtils.applyCommonConfig; +import static org.apache.nifi.parquet.utils.ParquetUtils.createParquetConfig; + +@Tags({"parquet", "parse", "record", "row", "reader"}) +@CapabilityDescription("Parses Parquet data and returns each Parquet record as a separate Record object. " + + "The schema will come from the Parquet data itself.") +public class ParquetReader extends AbstractControllerService implements RecordReaderFactory { + + @Override + public RecordReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) throws IOException { + final Configuration conf = new Configuration(); + final ParquetConfig parquetConfig = createParquetConfig(getConfigurationContext(), variables); + applyCommonConfig(conf, parquetConfig); + return new ParquetRecordReader(in, inputLength, conf); + } + + @Override + protected List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(); + properties.add(ParquetUtils.AVRO_READ_COMPATIBILITY); + return properties; + } +} diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/ParquetRecordSetWriter.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/ParquetRecordSetWriter.java new file mode 100644 index 000000000000..85af65fb589e --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/ParquetRecordSetWriter.java @@ -0,0 +1,116 @@ +/* + * 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.parquet; + +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.LoadingCache; +import org.apache.avro.Schema; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.avro.AvroTypeUtil; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.parquet.record.WriteParquetResult; +import org.apache.nifi.parquet.utils.ParquetConfig; +import org.apache.nifi.parquet.utils.ParquetUtils; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.RecordSetWriter; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.SchemaRegistryRecordSetWriter; +import org.apache.nifi.serialization.record.RecordSchema; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.nifi.parquet.utils.ParquetUtils.createParquetConfig; + +@Tags({"parquet", "result", "set", "writer", "serializer", "record", "recordset", "row"}) +@CapabilityDescription("Writes the contents of a RecordSet in Parquet format.") +public class ParquetRecordSetWriter extends SchemaRegistryRecordSetWriter implements RecordSetWriterFactory { + + public static final PropertyDescriptor CACHE_SIZE = new PropertyDescriptor.Builder() + .name("cache-size") + .displayName("Cache Size") + .description("Specifies how many Schemas should be cached") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .required(true) + .build(); + + private LoadingCache compiledAvroSchemaCache; + + @OnEnabled + public void onEnabled(final ConfigurationContext context) { + final int cacheSize = context.getProperty(CACHE_SIZE).asInteger(); + compiledAvroSchemaCache = Caffeine.newBuilder() + .maximumSize(cacheSize) + .build(schemaText -> new Schema.Parser().parse(schemaText)); + } + + @Override + public RecordSetWriter createWriter(final ComponentLog logger, final RecordSchema recordSchema, + final OutputStream out, final Map variables) throws IOException { + final ParquetConfig parquetConfig = createParquetConfig(getConfigurationContext(), variables); + try { + final Schema avroSchema; + try { + if (recordSchema.getSchemaFormat().isPresent() && recordSchema.getSchemaFormat().get().equals(AvroTypeUtil.AVRO_SCHEMA_FORMAT)) { + final Optional textOption = recordSchema.getSchemaText(); + if (textOption.isPresent()) { + avroSchema = compiledAvroSchemaCache.get(textOption.get()); + } else { + avroSchema = AvroTypeUtil.extractAvroSchema(recordSchema); + } + } else { + avroSchema = AvroTypeUtil.extractAvroSchema(recordSchema); + } + } catch (final Exception e) { + throw new SchemaNotFoundException("Failed to compile Avro Schema", e); + } + + return new WriteParquetResult(avroSchema, out, parquetConfig, logger); + + } catch (final SchemaNotFoundException e) { + throw new ProcessException("Could not determine the Avro Schema to use for writing the content", e); + } + } + + @Override + protected List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); + properties.add(CACHE_SIZE); + properties.add(ParquetUtils.COMPRESSION_TYPE); + properties.add(ParquetUtils.ROW_GROUP_SIZE); + properties.add(ParquetUtils.PAGE_SIZE); + properties.add(ParquetUtils.DICTIONARY_PAGE_SIZE); + properties.add(ParquetUtils.MAX_PADDING_SIZE); + properties.add(ParquetUtils.ENABLE_DICTIONARY_ENCODING); + properties.add(ParquetUtils.ENABLE_VALIDATION); + properties.add(ParquetUtils.WRITER_VERSION); + properties.add(ParquetUtils.AVRO_WRITE_OLD_LIST_STRUCTURE); + properties.add(ParquetUtils.AVRO_ADD_LIST_ELEMENT_RECORDS); + return properties; + } +} diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/record/AvroParquetHDFSRecordReader.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/hadoop/AvroParquetHDFSRecordReader.java similarity index 97% rename from nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/record/AvroParquetHDFSRecordReader.java rename to nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/hadoop/AvroParquetHDFSRecordReader.java index 8421e3756ee6..15c0cd554d5e 100644 --- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/record/AvroParquetHDFSRecordReader.java +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/hadoop/AvroParquetHDFSRecordReader.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.processors.parquet.record; +package org.apache.nifi.parquet.hadoop; import org.apache.avro.generic.GenericRecord; import org.apache.nifi.avro.AvroTypeUtil; diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/record/AvroParquetHDFSRecordWriter.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/hadoop/AvroParquetHDFSRecordWriter.java similarity index 97% rename from nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/record/AvroParquetHDFSRecordWriter.java rename to nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/hadoop/AvroParquetHDFSRecordWriter.java index 7ef37b1d9936..caa244c6f547 100644 --- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/record/AvroParquetHDFSRecordWriter.java +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/hadoop/AvroParquetHDFSRecordWriter.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.processors.parquet.record; +package org.apache.nifi.parquet.hadoop; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/record/ParquetRecordReader.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/record/ParquetRecordReader.java new file mode 100644 index 000000000000..6ff319d6b52a --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/record/ParquetRecordReader.java @@ -0,0 +1,98 @@ +/* + * 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.parquet.record; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.nifi.avro.AvroTypeUtil; +import org.apache.nifi.parquet.stream.NifiParquetInputFile; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.record.MapRecord; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.io.InputFile; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +public class ParquetRecordReader implements RecordReader { + + private GenericRecord lastParquetRecord; + private RecordSchema recordSchema; + + private final InputStream inputStream; + private final InputFile inputFile; + private final ParquetReader parquetReader; + + public ParquetRecordReader(final InputStream inputStream, final long inputLength, final Configuration configuration) throws IOException { + if (inputLength < 0) { + throw new IllegalArgumentException("Invalid input length of '" + inputLength + "'. This record reader requires knowing " + + "the length of the InputStream and cannot be used in some cases where the length may not be known."); + } + + this.inputStream = inputStream; + + inputFile = new NifiParquetInputFile(inputStream, inputLength); + parquetReader = AvroParquetReader.builder(inputFile).withConf(configuration).build(); + + // Read the first record so that we can extract the schema + lastParquetRecord = parquetReader.read(); + if (lastParquetRecord == null) { + throw new EOFException("Unable to obtain schema because no records were available"); + } + + // Convert Avro schema to RecordSchema + recordSchema = AvroTypeUtil.createSchema(lastParquetRecord.getSchema()); + } + + @Override + public Record nextRecord(boolean coerceTypes, boolean dropUnknownFields) throws IOException { + // If null then no more records are available + if (lastParquetRecord == null) { + return null; + } + + // Convert the last Parquet GenericRecord to NiFi Record + final Map values = AvroTypeUtil.convertAvroRecordToMap(lastParquetRecord, recordSchema); + final Record record = new MapRecord(recordSchema, values); + + // Read the next record and store for next time + lastParquetRecord = parquetReader.read(); + + // Return the converted record + return record; + } + + @Override + public RecordSchema getSchema() { + return recordSchema; + } + + @Override + public void close() throws IOException { + try { + parquetReader.close(); + } finally { + // ensure the input stream still gets closed + inputStream.close(); + } + } +} diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/record/WriteParquetResult.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/record/WriteParquetResult.java new file mode 100644 index 000000000000..9f7ce579cd5a --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/record/WriteParquetResult.java @@ -0,0 +1,81 @@ +/* + * 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.parquet.record; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.nifi.avro.AvroTypeUtil; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.parquet.stream.NifiParquetOutputFile; +import org.apache.nifi.parquet.utils.ParquetConfig; +import org.apache.nifi.serialization.AbstractRecordSetWriter; +import org.apache.nifi.serialization.record.Record; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.io.OutputFile; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Collections; +import java.util.Map; + +import static org.apache.nifi.parquet.utils.ParquetUtils.applyCommonConfig; + +public class WriteParquetResult extends AbstractRecordSetWriter { + + private final Schema schema; + private final ParquetWriter parquetWriter; + private final ComponentLog componentLogger; + + public WriteParquetResult(final Schema schema, final OutputStream out, final ParquetConfig parquetConfig, final ComponentLog componentLogger) throws IOException { + super(out); + this.schema = schema; + this.componentLogger = componentLogger; + + final Configuration conf = new Configuration(); + final OutputFile outputFile = new NifiParquetOutputFile(out); + + final AvroParquetWriter.Builder writerBuilder = + AvroParquetWriter.builder(outputFile).withSchema(schema); + applyCommonConfig(writerBuilder, conf, parquetConfig); + parquetWriter = writerBuilder.build(); + } + + @Override + protected Map writeRecord(final Record record) throws IOException { + final GenericRecord genericRecord = AvroTypeUtil.createAvroRecord(record, schema); + parquetWriter.write(genericRecord); + return Collections.emptyMap(); + } + + @Override + public void close() throws IOException { + try { + parquetWriter.close(); + } finally { + // ensure the output stream still gets closed + super.close(); + } + } + + @Override + public String getMimeType() { + return "application/parquet"; + } + +} diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/stream/NifiOutputStream.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiOutputStream.java similarity index 97% rename from nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/stream/NifiOutputStream.java rename to nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiOutputStream.java index acb2dc46a225..6212795566e9 100644 --- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/stream/NifiOutputStream.java +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiOutputStream.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.nifi.processors.parquet.stream; +package org.apache.nifi.parquet.stream; import org.apache.parquet.io.PositionOutputStream; diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiParquetInputFile.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiParquetInputFile.java new file mode 100644 index 000000000000..c4ac722c7b35 --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiParquetInputFile.java @@ -0,0 +1,53 @@ +/* + * 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.parquet.stream; + +import org.apache.nifi.stream.io.ByteCountingInputStream; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.SeekableInputStream; + +import java.io.IOException; +import java.io.InputStream; + +public class NifiParquetInputFile implements InputFile { + + private final long length; + private final ByteCountingInputStream input; + + public NifiParquetInputFile(final InputStream input, final long length) { + if (input == null) { + throw new IllegalArgumentException("InputStream is required"); + } + + if (!input.markSupported()) { + throw new IllegalArgumentException("InputStream must support mark/reset to be used with NifiParquetInputFile"); + } + + this.input = new ByteCountingInputStream(input); + this.length = length; + } + + @Override + public long getLength() throws IOException { + return length; + } + + @Override + public SeekableInputStream newStream() throws IOException { + return new NifiSeekableInputStream(input); + } +} diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/stream/NifiParquetOutputFile.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiParquetOutputFile.java similarity index 96% rename from nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/stream/NifiParquetOutputFile.java rename to nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiParquetOutputFile.java index d549b7bf11f2..2125e81a0935 100644 --- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/stream/NifiParquetOutputFile.java +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiParquetOutputFile.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.nifi.processors.parquet.stream; +package org.apache.nifi.parquet.stream; import org.apache.parquet.io.OutputFile; import org.apache.parquet.io.PositionOutputStream; diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiSeekableInputStream.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiSeekableInputStream.java new file mode 100644 index 000000000000..cd6b82053624 --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/stream/NifiSeekableInputStream.java @@ -0,0 +1,71 @@ +/* + * 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.parquet.stream; + +import org.apache.nifi.stream.io.ByteCountingInputStream; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.parquet.io.DelegatingSeekableInputStream; + +import java.io.IOException; + +public class NifiSeekableInputStream extends DelegatingSeekableInputStream { + + private final ByteCountingInputStream input; + + public NifiSeekableInputStream(final ByteCountingInputStream input) { + super(input); + this.input = input; + this.input.mark(Integer.MAX_VALUE); + } + + @Override + public long getPos() throws IOException { + return input.getBytesConsumed(); + } + + @Override + public void seek(long newPos) throws IOException { + final long currentPos = getPos(); + if (newPos == currentPos) { + return; + } + + if (newPos < currentPos) { + // seeking backwards so first reset back to beginning of the stream then seek + input.reset(); + input.mark(Integer.MAX_VALUE); + } + + // must call getPos() again in case reset was called above + StreamUtils.skip(input, newPos - getPos()); + } + + @Override + public boolean markSupported() { + return false; + } + + @Override + public synchronized void mark(int readlimit) { + throw new UnsupportedOperationException("Mark/reset is not supported"); + } + + @Override + public synchronized void reset() throws IOException { + throw new UnsupportedOperationException("Mark/reset is not supported"); + } +} diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/utils/ParquetConfig.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/utils/ParquetConfig.java new file mode 100644 index 000000000000..5cc4abe2987c --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/utils/ParquetConfig.java @@ -0,0 +1,133 @@ +/* + * 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.parquet.utils; + +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +public class ParquetConfig { + + private Integer rowGroupSize; + private Integer pageSize; + private Integer dictionaryPageSize; + private Integer maxPaddingSize; + private Boolean enableDictionaryEncoding; + private Boolean enableValidation; + private Boolean avroReadCompatibility; + private Boolean avroAddListElementRecords; + private Boolean avroWriteOldListStructure; + private ParquetProperties.WriterVersion writerVersion; + private ParquetFileWriter.Mode writerMode; + private CompressionCodecName compressionCodec; + + public Integer getRowGroupSize() { + return rowGroupSize; + } + + public void setRowGroupSize(Integer rowGroupSize) { + this.rowGroupSize = rowGroupSize; + } + + public Integer getPageSize() { + return pageSize; + } + + public void setPageSize(Integer pageSize) { + this.pageSize = pageSize; + } + + public Integer getDictionaryPageSize() { + return dictionaryPageSize; + } + + public void setDictionaryPageSize(Integer dictionaryPageSize) { + this.dictionaryPageSize = dictionaryPageSize; + } + + public Integer getMaxPaddingSize() { + return maxPaddingSize; + } + + public void setMaxPaddingSize(Integer maxPaddingSize) { + this.maxPaddingSize = maxPaddingSize; + } + + public Boolean getEnableDictionaryEncoding() { + return enableDictionaryEncoding; + } + + public void setEnableDictionaryEncoding(Boolean enableDictionaryEncoding) { + this.enableDictionaryEncoding = enableDictionaryEncoding; + } + + public Boolean getEnableValidation() { + return enableValidation; + } + + public void setEnableValidation(Boolean enableValidation) { + this.enableValidation = enableValidation; + } + + public Boolean getAvroReadCompatibility() { + return avroReadCompatibility; + } + + public void setAvroReadCompatibility(Boolean avroReadCompatibility) { + this.avroReadCompatibility = avroReadCompatibility; + } + + public Boolean getAvroAddListElementRecords() { + return avroAddListElementRecords; + } + + public void setAvroAddListElementRecords(Boolean avroAddListElementRecords) { + this.avroAddListElementRecords = avroAddListElementRecords; + } + + public Boolean getAvroWriteOldListStructure() { + return avroWriteOldListStructure; + } + + public void setAvroWriteOldListStructure(Boolean avroWriteOldListStructure) { + this.avroWriteOldListStructure = avroWriteOldListStructure; + } + + public ParquetProperties.WriterVersion getWriterVersion() { + return writerVersion; + } + + public void setWriterVersion(ParquetProperties.WriterVersion writerVersion) { + this.writerVersion = writerVersion; + } + + public ParquetFileWriter.Mode getWriterMode() { + return writerMode; + } + + public void setWriterMode(ParquetFileWriter.Mode writerMode) { + this.writerMode = writerMode; + } + + public CompressionCodecName getCompressionCodec() { + return compressionCodec; + } + + public void setCompressionCodec(CompressionCodecName compressionCodec) { + this.compressionCodec = compressionCodec; + } +} diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/utils/ParquetUtils.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/utils/ParquetUtils.java similarity index 55% rename from nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/utils/ParquetUtils.java rename to nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/utils/ParquetUtils.java index 7b116c233eea..0f00df8e9968 100644 --- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/utils/ParquetUtils.java +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/parquet/utils/ParquetUtils.java @@ -16,25 +16,28 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.nifi.processors.parquet.utils; +package org.apache.nifi.parquet.utils; import org.apache.hadoop.conf.Configuration; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.context.PropertyContext; import org.apache.nifi.expression.ExpressionLanguageScope; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.DataUnit; -import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.parquet.PutParquet; +import org.apache.parquet.avro.AvroReadSupport; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.avro.AvroWriteSupport; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.hadoop.ParquetFileWriter; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; + import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; public class ParquetUtils { @@ -96,18 +99,36 @@ public class ParquetUtils { .allowableValues(org.apache.parquet.column.ParquetProperties.WriterVersion.values()) .build(); - public static List COMPRESSION_TYPES = getCompressionTypes(); + public static final PropertyDescriptor AVRO_READ_COMPATIBILITY = new PropertyDescriptor.Builder() + .name("avro-read-compatibility") + .displayName("Avro Read Compatibility") + .description("Specifies the value for '" + AvroReadSupport.AVRO_COMPATIBILITY + "' in the underlying Parquet library") + .allowableValues("true", "false") + .defaultValue("true") + .required(true) + .build(); - public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() - .name("compression-type") - .displayName("Compression Type") - .description("The type of compression for the file being written.") - .allowableValues(COMPRESSION_TYPES.toArray(new AllowableValue[0])) - .defaultValue(COMPRESSION_TYPES.get(0).getValue()) + public static final PropertyDescriptor AVRO_ADD_LIST_ELEMENT_RECORDS = new PropertyDescriptor.Builder() + .name("avro-add-list-element-records") + .displayName("Avro Add List Element Records") + .description("Specifies the value for '" + AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS + "' in the underlying Parquet library") + .allowableValues("true", "false") + .defaultValue("true") + .required(true) + .build(); + + public static final PropertyDescriptor AVRO_WRITE_OLD_LIST_STRUCTURE = new PropertyDescriptor.Builder() + .name("avro-write-old-list-structure") + .displayName("Avro Write Old List Structure") + .description("Specifies the value for '" + AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE + "' in the underlying Parquet library") + .allowableValues("true", "false") + .defaultValue("true") .required(true) .build(); - public static List getCompressionTypes() { + public static final List COMPRESSION_TYPES = getCompressionTypes(); + + private static List getCompressionTypes() { final List compressionTypes = new ArrayList<>(); for (CompressionCodecName compressionCodecName : CompressionCodecName.values()) { final String name = compressionCodecName.name(); @@ -116,35 +137,48 @@ public static List getCompressionTypes() { return Collections.unmodifiableList(compressionTypes); } - public static void applyCommonConfig(final ParquetWriter.Builder builder, - final ProcessContext context, - final FlowFile flowFile, - final Configuration conf, - final AbstractProcessor abstractProcessor) { - builder.withConf(conf); + // NOTE: This needs to be named the same as the compression property in AbstractPutHDFSRecord + public static final String COMPRESSION_TYPE_PROP_NAME = "compression-type"; + + public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() + .name(COMPRESSION_TYPE_PROP_NAME) + .displayName("Compression Type") + .description("The type of compression for the file being written.") + .allowableValues(COMPRESSION_TYPES.toArray(new AllowableValue[0])) + .defaultValue(COMPRESSION_TYPES.get(0).getValue()) + .required(true) + .build(); + + /** + * Creates a ParquetConfig instance from the given PropertyContext. + * + * @param context the PropertyContext from a component + * @param variables an optional set of variables to evaluate EL against, may be null + * @return the ParquetConfig + */ + public static ParquetConfig createParquetConfig(final PropertyContext context, final Map variables) { + final ParquetConfig parquetConfig = new ParquetConfig(); // Required properties boolean overwrite = true; - if(context.getProperty(PutParquet.OVERWRITE).isSet()) + if(context.getProperty(PutParquet.OVERWRITE).isSet()) { overwrite = context.getProperty(PutParquet.OVERWRITE).asBoolean(); + } final ParquetFileWriter.Mode mode = overwrite ? ParquetFileWriter.Mode.OVERWRITE : ParquetFileWriter.Mode.CREATE; - builder.withWriteMode(mode); - - final PropertyDescriptor compressionTypeDescriptor = abstractProcessor.getPropertyDescriptor(COMPRESSION_TYPE.getName()); - - final String compressionTypeValue = context.getProperty(compressionTypeDescriptor).getValue(); + parquetConfig.setWriterMode(mode); + final String compressionTypeValue = context.getProperty(ParquetUtils.COMPRESSION_TYPE).getValue(); final CompressionCodecName codecName = CompressionCodecName.valueOf(compressionTypeValue); - builder.withCompressionCodec(codecName); + parquetConfig.setCompressionCodec(codecName); // Optional properties if (context.getProperty(ROW_GROUP_SIZE).isSet()){ try { - final Double rowGroupSize = context.getProperty(ROW_GROUP_SIZE).evaluateAttributeExpressions(flowFile).asDataSize(DataUnit.B); + final Double rowGroupSize = context.getProperty(ROW_GROUP_SIZE).evaluateAttributeExpressions(variables).asDataSize(DataUnit.B); if (rowGroupSize != null) { - builder.withRowGroupSize(rowGroupSize.intValue()); + parquetConfig.setRowGroupSize(rowGroupSize.intValue()); } } catch (IllegalArgumentException e) { throw new IllegalArgumentException("Invalid data size for " + ROW_GROUP_SIZE.getDisplayName(), e); @@ -153,9 +187,9 @@ public static void applyCommonConfig(final ParquetWriter.Builder builder, if (context.getProperty(PAGE_SIZE).isSet()) { try { - final Double pageSize = context.getProperty(PAGE_SIZE).evaluateAttributeExpressions(flowFile).asDataSize(DataUnit.B); + final Double pageSize = context.getProperty(PAGE_SIZE).evaluateAttributeExpressions(variables).asDataSize(DataUnit.B); if (pageSize != null) { - builder.withPageSize(pageSize.intValue()); + parquetConfig.setPageSize(pageSize.intValue()); } } catch (IllegalArgumentException e) { throw new IllegalArgumentException("Invalid data size for " + PAGE_SIZE.getDisplayName(), e); @@ -164,9 +198,9 @@ public static void applyCommonConfig(final ParquetWriter.Builder builder, if (context.getProperty(DICTIONARY_PAGE_SIZE).isSet()) { try { - final Double dictionaryPageSize = context.getProperty(DICTIONARY_PAGE_SIZE).evaluateAttributeExpressions(flowFile).asDataSize(DataUnit.B); + final Double dictionaryPageSize = context.getProperty(DICTIONARY_PAGE_SIZE).evaluateAttributeExpressions(variables).asDataSize(DataUnit.B); if (dictionaryPageSize != null) { - builder.withDictionaryPageSize(dictionaryPageSize.intValue()); + parquetConfig.setDictionaryPageSize(dictionaryPageSize.intValue()); } } catch (IllegalArgumentException e) { throw new IllegalArgumentException("Invalid data size for " + DICTIONARY_PAGE_SIZE.getDisplayName(), e); @@ -175,9 +209,9 @@ public static void applyCommonConfig(final ParquetWriter.Builder builder, if (context.getProperty(MAX_PADDING_SIZE).isSet()) { try { - final Double maxPaddingSize = context.getProperty(MAX_PADDING_SIZE).evaluateAttributeExpressions(flowFile).asDataSize(DataUnit.B); + final Double maxPaddingSize = context.getProperty(MAX_PADDING_SIZE).evaluateAttributeExpressions(variables).asDataSize(DataUnit.B); if (maxPaddingSize != null) { - builder.withMaxPaddingSize(maxPaddingSize.intValue()); + parquetConfig.setMaxPaddingSize(maxPaddingSize.intValue()); } } catch (IllegalArgumentException e) { throw new IllegalArgumentException("Invalid data size for " + MAX_PADDING_SIZE.getDisplayName(), e); @@ -186,17 +220,93 @@ public static void applyCommonConfig(final ParquetWriter.Builder builder, if (context.getProperty(ENABLE_DICTIONARY_ENCODING).isSet()) { final boolean enableDictionaryEncoding = context.getProperty(ENABLE_DICTIONARY_ENCODING).asBoolean(); - builder.withDictionaryEncoding(enableDictionaryEncoding); + parquetConfig.setEnableDictionaryEncoding(enableDictionaryEncoding); } if (context.getProperty(ENABLE_VALIDATION).isSet()) { final boolean enableValidation = context.getProperty(ENABLE_VALIDATION).asBoolean(); - builder.withValidation(enableValidation); + parquetConfig.setEnableValidation(enableValidation); } if (context.getProperty(WRITER_VERSION).isSet()) { final String writerVersionValue = context.getProperty(WRITER_VERSION).getValue(); - builder.withWriterVersion(ParquetProperties.WriterVersion.valueOf(writerVersionValue)); + parquetConfig.setWriterVersion(ParquetProperties.WriterVersion.valueOf(writerVersionValue)); + } + + if (context.getProperty(AVRO_READ_COMPATIBILITY).isSet()) { + final boolean avroReadCompatibility = context.getProperty(AVRO_READ_COMPATIBILITY).asBoolean(); + parquetConfig.setAvroReadCompatibility(avroReadCompatibility); + } + + if (context.getProperty(AVRO_ADD_LIST_ELEMENT_RECORDS).isSet()) { + final boolean avroAddListElementRecords = context.getProperty(AVRO_ADD_LIST_ELEMENT_RECORDS).asBoolean(); + parquetConfig.setAvroAddListElementRecords(avroAddListElementRecords); + } + + if (context.getProperty(AVRO_WRITE_OLD_LIST_STRUCTURE).isSet()) { + final boolean avroWriteOldListStructure = context.getProperty(AVRO_WRITE_OLD_LIST_STRUCTURE).asBoolean(); + parquetConfig.setAvroWriteOldListStructure(avroWriteOldListStructure); + } + + return parquetConfig; + } + + public static void applyCommonConfig(final ParquetWriter.Builder builder, final Configuration conf, + final ParquetConfig parquetConfig) { + builder.withConf(conf); + builder.withCompressionCodec(parquetConfig.getCompressionCodec()); + + // Optional properties + + if (parquetConfig.getRowGroupSize() != null){ + builder.withRowGroupSize(parquetConfig.getRowGroupSize()); + } + + if (parquetConfig.getPageSize() != null) { + builder.withPageSize(parquetConfig.getPageSize()); + } + + if (parquetConfig.getDictionaryPageSize() != null) { + builder.withDictionaryPageSize(parquetConfig.getDictionaryPageSize()); + } + + if (parquetConfig.getMaxPaddingSize() != null) { + builder.withMaxPaddingSize(parquetConfig.getMaxPaddingSize()); + } + + if (parquetConfig.getEnableDictionaryEncoding() != null) { + builder.withDictionaryEncoding(parquetConfig.getEnableDictionaryEncoding()); + } + + if (parquetConfig.getEnableValidation() != null) { + builder.withValidation(parquetConfig.getEnableValidation()); + } + + if (parquetConfig.getWriterVersion() != null) { + builder.withWriterVersion(parquetConfig.getWriterVersion()); + } + + if (parquetConfig.getWriterMode() != null) { + builder.withWriteMode(parquetConfig.getWriterMode()); + } + + applyCommonConfig(conf, parquetConfig); + } + + public static void applyCommonConfig(Configuration conf, ParquetConfig parquetConfig) { + if (parquetConfig.getAvroReadCompatibility() != null) { + conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, + parquetConfig.getAvroReadCompatibility().booleanValue()); + } + + if (parquetConfig.getAvroAddListElementRecords() != null) { + conf.setBoolean(AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS, + parquetConfig.getAvroAddListElementRecords().booleanValue()); + } + + if (parquetConfig.getAvroWriteOldListStructure() != null) { + conf.setBoolean(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, + parquetConfig.getAvroWriteOldListStructure().booleanValue()); } } } diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/ConvertAvroToParquet.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/ConvertAvroToParquet.java index 0bcf606f227e..eec48afd0e1d 100644 --- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/ConvertAvroToParquet.java +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/ConvertAvroToParquet.java @@ -33,29 +33,33 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.parquet.stream.NifiParquetOutputFile; +import org.apache.nifi.parquet.utils.ParquetConfig; +import org.apache.nifi.parquet.utils.ParquetUtils; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processors.parquet.stream.NifiParquetOutputFile; -import org.apache.nifi.processors.parquet.utils.ParquetUtils; import org.apache.parquet.avro.AvroParquetWriter; -import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.ParquetWriter; -import java.io.InputStream; -import java.io.OutputStream; + import java.io.BufferedInputStream; import java.io.IOException; -import java.util.Map; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.ArrayList; +import java.util.Map; import java.util.Set; -import java.util.Collections; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.nifi.parquet.utils.ParquetUtils.applyCommonConfig; +import static org.apache.nifi.parquet.utils.ParquetUtils.createParquetConfig; + @Tags({"avro", "parquet", "convert"}) @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) @CapabilityDescription("Converts Avro records into Parquet file format. The incoming FlowFile should be a valid avro file. If an incoming FlowFile does " @@ -191,12 +195,13 @@ private ParquetWriter createParquetWriter(final ProcessContext context, final Fl .builder(nifiParquetOutputFile) .withSchema(schema); - Configuration conf = new Configuration(); - conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, true); - conf.setBoolean("parquet.avro.add-list-element-records", false); - conf.setBoolean("parquet.avro.write-old-list-structure", false); + final ParquetConfig parquetConfig = createParquetConfig(context, flowFile.getAttributes()); + parquetConfig.setAvroReadCompatibility(true); + parquetConfig.setAvroAddListElementRecords(false); + parquetConfig.setAvroWriteOldListStructure(false); - ParquetUtils.applyCommonConfig(parquetWriter, context, flowFile, conf, this); + final Configuration conf = new Configuration(); + applyCommonConfig(parquetWriter, conf, parquetConfig); return parquetWriter.build(); } diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/FetchParquet.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/FetchParquet.java index f4a6875327de..aa1eabd65439 100644 --- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/FetchParquet.java +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/FetchParquet.java @@ -33,7 +33,7 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processors.hadoop.AbstractFetchHDFSRecord; import org.apache.nifi.processors.hadoop.record.HDFSRecordReader; -import org.apache.nifi.processors.parquet.record.AvroParquetHDFSRecordReader; +import org.apache.nifi.parquet.hadoop.AvroParquetHDFSRecordReader; import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.hadoop.ParquetReader; import java.io.IOException; diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/PutParquet.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/PutParquet.java index c2794ac039f6..d7d55c227b51 100644 --- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/PutParquet.java +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/java/org/apache/nifi/processors/parquet/PutParquet.java @@ -33,13 +33,14 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.RequiredPermission; import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.parquet.utils.ParquetConfig; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processors.hadoop.AbstractPutHDFSRecord; import org.apache.nifi.processors.hadoop.record.HDFSRecordWriter; -import org.apache.nifi.processors.parquet.record.AvroParquetHDFSRecordWriter; -import org.apache.nifi.processors.parquet.utils.ParquetUtils; +import org.apache.nifi.parquet.hadoop.AvroParquetHDFSRecordWriter; +import org.apache.nifi.parquet.utils.ParquetUtils; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.record.RecordSchema; import org.apache.parquet.avro.AvroParquetWriter; @@ -49,6 +50,9 @@ import java.util.Collections; import java.util.List; +import static org.apache.nifi.parquet.utils.ParquetUtils.createParquetConfig; +import static org.apache.nifi.parquet.utils.ParquetUtils.applyCommonConfig; + @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) @Tags({"put", "parquet", "hadoop", "HDFS", "filesystem", "record"}) @CapabilityDescription("Reads records from an incoming FlowFile using the provided Record Reader, and writes those records " + @@ -79,19 +83,9 @@ public class PutParquet extends AbstractPutHDFSRecord { .defaultValue("false") .build(); - public static final List COMPRESSION_TYPES; - static { - final List compressionTypes = new ArrayList<>(); - for (CompressionCodecName compressionCodecName : CompressionCodecName.values()) { - final String name = compressionCodecName.name(); - compressionTypes.add(new AllowableValue(name, name)); - } - COMPRESSION_TYPES = Collections.unmodifiableList(compressionTypes); - } - @Override public List getCompressionTypes(final ProcessorInitializationContext context) { - return COMPRESSION_TYPES; + return ParquetUtils.COMPRESSION_TYPES; } @Override @@ -109,6 +103,8 @@ public List getAdditionalProperties() { props.add(ParquetUtils.ENABLE_DICTIONARY_ENCODING); props.add(ParquetUtils.ENABLE_VALIDATION); props.add(ParquetUtils.WRITER_VERSION); + props.add(ParquetUtils.AVRO_WRITE_OLD_LIST_STRUCTURE); + props.add(ParquetUtils.AVRO_ADD_LIST_ELEMENT_RECORDS); props.add(REMOVE_CRC_FILES); return Collections.unmodifiableList(props); } @@ -123,7 +119,8 @@ public HDFSRecordWriter createHDFSRecordWriter(final ProcessContext context, fin .builder(path) .withSchema(avroSchema); - ParquetUtils.applyCommonConfig(parquetWriter, context, flowFile, conf, this); + final ParquetConfig parquetConfig = createParquetConfig(context, flowFile.getAttributes()); + applyCommonConfig(parquetWriter, conf, parquetConfig); return new AvroParquetHDFSRecordWriter(parquetWriter.build(), avroSchema); } diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService new file mode 100644 index 000000000000..6b009e1d5711 --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -0,0 +1,17 @@ +# 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. + +org.apache.nifi.parquet.ParquetRecordSetWriter +org.apache.nifi.parquet.ParquetReader \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetReader.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetReader.java new file mode 100644 index 000000000000..da5424217d85 --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetReader.java @@ -0,0 +1,119 @@ +/* + * 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.parquet; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.util.MockComponentLog; +import org.apache.nifi.util.MockConfigurationContext; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class TestParquetReader { + + private Map readerFactoryProperties; + private ConfigurationContext readerFactoryConfigContext; + + private ParquetReader parquetReaderFactory; + private ComponentLog componentLog; + + @Before + public void setup() { + readerFactoryProperties = new HashMap<>(); + readerFactoryConfigContext = new MockConfigurationContext(readerFactoryProperties, null); + + parquetReaderFactory = new ParquetReader(); + parquetReaderFactory.abstractStoreConfigContext(readerFactoryConfigContext); + + componentLog = new MockComponentLog("1234", parquetReaderFactory); + } + + @Test + public void testReadUsers() throws IOException, MalformedRecordException { + final Schema schema = getSchema("src/test/resources/avro/user.avsc"); + final File parquetFile = new File("target/TestParquetReader-testReadUsers-" + System.currentTimeMillis()); + + // write some users to the parquet file... + final int numUsers = 10; + try (final ParquetWriter writer = createParquetWriter(schema, parquetFile)) { + for (int i=0; i < numUsers; i++) { + final GenericRecord user = new GenericData.Record(schema); + user.put("name", "Bob" + i); + user.put("favorite_number", i); + user.put("favorite_color", "blue" + i); + writer.write(user); + } + } + + // read the parquet file into bytes since we can't use a FileInputStream since it doesn't support mark/reset + final byte[] parquetBytes = IOUtils.toByteArray(parquetFile.toURI()); + + // read the users in using the record reader... + try (final InputStream in = new ByteArrayInputStream(parquetBytes); + final RecordReader recordReader = parquetReaderFactory.createRecordReader( + Collections.emptyMap(), in, parquetFile.length(), componentLog)) { + + int recordCount = 0; + while (recordReader.nextRecord() != null) { + recordCount++; + } + assertEquals(numUsers, recordCount); + } + } + + private Schema getSchema(final String schemaFilePath) throws IOException { + final File schemaFile = new File(schemaFilePath); + final String schemaString = IOUtils.toString(new FileInputStream(schemaFile), StandardCharsets.UTF_8); + return new Schema.Parser().parse(schemaString); + } + + private ParquetWriter createParquetWriter(final Schema schema, final File parquetFile) throws IOException { + final Configuration conf = new Configuration(); + final Path parquetPath = new Path(parquetFile.getPath()); + + final ParquetWriter writer = + AvroParquetWriter.builder(parquetPath) + .withSchema(schema) + .withConf(conf) + .build(); + + return writer; + } +} diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetRecordSetWriter.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetRecordSetWriter.java new file mode 100644 index 000000000000..637ba227f636 --- /dev/null +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetRecordSetWriter.java @@ -0,0 +1,159 @@ +/* + * 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.parquet; + +import org.apache.avro.generic.GenericRecord; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaAccessUtils; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.RecordSetWriter; +import org.apache.nifi.serialization.SimpleRecordSchema; +import org.apache.nifi.serialization.record.MapRecord; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; +import org.apache.nifi.util.MockComponentLog; +import org.apache.nifi.util.MockConfigurationContext; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.io.InputFile; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class TestParquetRecordSetWriter { + + private ComponentLog componentLog; + private ParquetRecordSetWriter recordSetWriterFactory; + + @Before + public void setup() { + recordSetWriterFactory = new ParquetRecordSetWriter(); + componentLog = new MockComponentLog("1234", recordSetWriterFactory); + } + + @Test + public void testWriteUsers() throws IOException, SchemaNotFoundException { + initRecordSetWriter("src/test/resources/avro/user.avsc"); + + // get the schema from the writer factory + final RecordSchema writeSchema = recordSetWriterFactory.getSchema(Collections.emptyMap(), null); + + // write some records + final int numUsers = 10; + final File parquetFile = new File("target/testWriterUsers-" + System.currentTimeMillis()); + + // write some records... + writeUsers(writeSchema, parquetFile, numUsers); + + // read the records back in to verify + verifyParquetRecords(parquetFile, numUsers); + } + + @Test + public void testWriteUsersWhenSchemaFormatNotAvro() throws IOException, SchemaNotFoundException { + initRecordSetWriter("src/test/resources/avro/user.avsc"); + + // get the schema from the writer factory + final RecordSchema writeSchema = recordSetWriterFactory.getSchema(Collections.emptyMap(), null); + final RecordSchema writeSchemaWithOtherFormat = new SimpleRecordSchema(writeSchema.getFields(), null, "OTHER-FORMAT", SchemaIdentifier.EMPTY); + + // write some records + final int numUsers = 10; + final File parquetFile = new File("target/testWriterUsers-" + System.currentTimeMillis()); + + // write some records... + writeUsers(writeSchemaWithOtherFormat, parquetFile, numUsers); + + // read the records back in to verify + verifyParquetRecords(parquetFile, numUsers); + } + + private void initRecordSetWriter(final String schemaFile) throws IOException { + final ConfigurationContext configurationContext = getConfigurationContextWithSchema(schemaFile); + + // simulate enabling the service + recordSetWriterFactory.onEnabled(configurationContext); + recordSetWriterFactory.storeSchemaWriteStrategy(configurationContext); + recordSetWriterFactory.storeSchemaAccessStrategy(configurationContext); + } + + private void writeUsers(final RecordSchema writeSchema, final File parquetFile, final int numUsers) throws IOException { + try(final OutputStream output = new FileOutputStream(parquetFile); + final RecordSetWriter recordSetWriter = recordSetWriterFactory.createWriter(componentLog, writeSchema, output, Collections.emptyMap())) { + for (int i = 0; i < numUsers; i++) { + final Map userFields = new HashMap<>(); + userFields.put("name", "user" + i); + userFields.put("favorite_number", i); + userFields.put("favorite_color", "blue"); + + final Record userRecord = new MapRecord(writeSchema, userFields); + recordSetWriter.write(userRecord); + } + + recordSetWriter.flush(); + } + } + + private void verifyParquetRecords(final File parquetFile, final int expectedRecordCount) throws IOException { + final Configuration conf = new Configuration(); + final Path path = new Path(parquetFile.getPath()); + final InputFile inputFile = HadoopInputFile.fromPath(path, conf); + + try (final ParquetReader reader = + AvroParquetReader.builder(inputFile).withConf(conf).build()){ + + int recordCount = 0; + while(reader.read() != null) { + recordCount++; + } + assertEquals(expectedRecordCount, recordCount); + } + } + + private ConfigurationContext getConfigurationContextWithSchema(String schemaPath) throws IOException { + final File schemaFile = new File(schemaPath); + final Map properties = createPropertiesWithSchema(schemaFile); + return new MockConfigurationContext(properties, null); + } + + private Map createPropertiesWithSchema(final File schemaFile) throws IOException { + return createPropertiesWithSchema(IOUtils.toString(schemaFile.toURI())); + } + + private Map createPropertiesWithSchema(final String schemaText) { + final Map propertyValues = new HashMap<>(); + propertyValues.put(SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY.getValue()); + propertyValues.put(SchemaAccessUtils.SCHEMA_TEXT, schemaText); + return propertyValues; + } +} diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/PutParquetTest.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/PutParquetTest.java index 84ff4345e5f7..c1bcbc7aecf9 100644 --- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/PutParquetTest.java +++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/PutParquetTest.java @@ -42,7 +42,7 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processors.hadoop.exception.FailureException; import org.apache.nifi.processors.hadoop.record.HDFSRecordWriter; -import org.apache.nifi.processors.parquet.utils.ParquetUtils; +import org.apache.nifi.parquet.utils.ParquetUtils; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventType; import org.apache.nifi.reporting.InitializationException; diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/record/script/ScriptedReader.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/record/script/ScriptedReader.java index 5983f183eedd..ac9f2b7722cf 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/record/script/ScriptedReader.java +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/record/script/ScriptedReader.java @@ -60,10 +60,10 @@ public void onEnabled(final ConfigurationContext context) { } @Override - public RecordReader createRecordReader(Map variables, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException { + public RecordReader createRecordReader(Map variables, InputStream in, long inputLength, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException { if (recordFactory.get() != null) { try { - return recordFactory.get().createRecordReader(variables, in, logger); + return recordFactory.get().createRecordReader(variables, in, inputLength, logger); } catch (UndeclaredThrowableException ute) { throw new IOException(ute.getCause()); } diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/groovy/org/apache/nifi/record/script/ScriptedReaderTest.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/groovy/org/apache/nifi/record/script/ScriptedReaderTest.groovy index 6142ddd7cf19..8be0b6f2d661 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/groovy/org/apache/nifi/record/script/ScriptedReaderTest.groovy +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/groovy/org/apache/nifi/record/script/ScriptedReaderTest.groovy @@ -97,9 +97,10 @@ class ScriptedReaderTest { recordReaderFactory.initialize initContext recordReaderFactory.onEnabled configurationContext - InputStream inStream = new ByteArrayInputStream('Flow file content not used'.bytes) + byte[] contentBytes = 'Flow file content not used'.bytes + InputStream inStream = new ByteArrayInputStream(contentBytes) - RecordReader recordReader = recordReaderFactory.createRecordReader(Collections.emptyMap(), inStream, logger) + RecordReader recordReader = recordReaderFactory.createRecordReader(Collections.emptyMap(), inStream, contentBytes.length, logger) assertNotNull(recordReader) 3.times { @@ -157,7 +158,7 @@ class ScriptedReaderTest { Map schemaVariables = ['record.tag': 'myRecord'] - InputStream inStream = new ByteArrayInputStream(''' + byte[] contentBytes = ''' 1 @@ -175,9 +176,11 @@ class ScriptedReaderTest { 300 - '''.bytes) + '''.bytes - RecordReader recordReader = recordReaderFactory.createRecordReader(schemaVariables, inStream, logger) + InputStream inStream = new ByteArrayInputStream(contentBytes) + + RecordReader recordReader = recordReaderFactory.createRecordReader(schemaVariables, inStream, contentBytes.length, logger) assertNotNull(recordReader) 3.times { diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_reader_inline.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_reader_inline.groovy index 83e772062f8e..783f5202a561 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_reader_inline.groovy +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_reader_inline.groovy @@ -57,7 +57,7 @@ class GroovyRecordReader implements RecordReader { class GroovyRecordReaderFactory extends AbstractControllerService implements RecordReaderFactory { - RecordReader createRecordReader(Map variables, InputStream inputStream, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException { + RecordReader createRecordReader(Map variables, InputStream inputStream, long inputLength, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException { return new GroovyRecordReader() } } diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_reader_xml.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_reader_xml.groovy index c7296099ad85..db2c03b07949 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_reader_xml.groovy +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_reader_xml.groovy @@ -67,7 +67,7 @@ class GroovyXmlRecordReaderFactory extends AbstractControllerService implements // Will be set by the ScriptedRecordReaderFactory ConfigurationContext configurationContext - RecordReader createRecordReader(Map variables, InputStream inputStream, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException { + RecordReader createRecordReader(Map variables, InputStream inputStream, long inputLength, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException { // Expecting 'schema.text' to have be an JSON array full of objects whose keys are the field name and the value maps to a RecordFieldType def schemaText = configurationContext.properties.find {p -> p.key.dynamic && p.key.name == 'schema.text'}?.getValue() if (!schemaText) return null diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java index f34f58f836cc..17e69277de2e 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java @@ -123,7 +123,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session @Override public void process(final InputStream in, final OutputStream out) throws IOException { - try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, getLogger())) { + try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { // Get the first record and process it before we create the Record Writer. We do this so that if the Processor // updates the Record's schema, we can provide an updated schema to the Record Writer. If there are no records, diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRouteRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRouteRecord.java index 46acf983734f..1a31d90eae71 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRouteRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRouteRecord.java @@ -121,7 +121,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session session.read(flowFile, new InputStreamCallback() { @Override public void process(final InputStream in) throws IOException { - try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, getLogger())) { + try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { final Record firstRecord = reader.nextRecord(); if (firstRecord == null) { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ForkRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ForkRecord.java index 1b719abea0be..5d69ea6136a8 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ForkRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ForkRecord.java @@ -17,20 +17,6 @@ package org.apache.nifi.processors.standard; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; - import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; @@ -73,6 +59,20 @@ import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.type.ArrayDataType; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; + @SideEffectFree @SupportsBatching @InputRequirement(Requirement.INPUT_REQUIRED) @@ -238,7 +238,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session session.read(flowFile, new InputStreamCallback() { @Override public void process(final InputStream in) throws IOException { - try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, getLogger())) { + try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, reader.getSchema()); final OutputStream out = session.write(outFlowFile); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenTCPRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenTCPRecord.java index 5ea3e6f32406..c6b6af9ab1f3 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenTCPRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenTCPRecord.java @@ -334,10 +334,10 @@ public void onTrigger(final ProcessContext context, final ProcessSession session synchronized (socketRecordReader) { FlowFile flowFile = session.create(); try { - // lazily creating the record reader here b/c we need a flow file, eventually shouldn't have to do this + // lazily creating the record reader here RecordReader recordReader = socketRecordReader.getRecordReader(); if (recordReader == null) { - recordReader = socketRecordReader.createRecordReader(flowFile, getLogger()); + recordReader = socketRecordReader.createRecordReader(getLogger()); } Record record; diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDPRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDPRecord.java index d331a332d474..6a723ea5cc82 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDPRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDPRecord.java @@ -242,7 +242,8 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final RecordReader reader; final List records = new ArrayList<>(); try (final InputStream in = new ByteArrayInputStream(event.getData())) { - reader = readerFactory.createRecordReader(Collections.emptyMap(), in, getLogger()); + final long inputLength = event.getData() == null ? -1 : event.getData().length; + reader = readerFactory.createRecordReader(Collections.emptyMap(), in, inputLength, getLogger()); Record record; while((record = reader.nextRecord()) != null) { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java index 6c0ef0844e4d..019849ce2085 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java @@ -17,22 +17,6 @@ package org.apache.nifi.processors.standard; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.Stream; - import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.InputRequirement; @@ -68,6 +52,22 @@ import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.util.DataTypeUtils; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; + @EventDriven @SupportsBatching @InputRequirement(Requirement.INPUT_REQUIRED) @@ -200,7 +200,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session try (final InputStream in = session.read(flowFile)) { final Map originalAttributes = flowFile.getAttributes(); - final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, getLogger()); + final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, flowFile.getSize(), getLogger()); final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, reader.getSchema()); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java index 514d0438b228..5136e67d973c 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java @@ -286,7 +286,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final RecordSchema readerSchema; try (final InputStream rawIn = session.read(original)) { final Map originalAttributes = original.getAttributes(); - final RecordReader reader = recordReaderFactory.createRecordReader(originalAttributes, rawIn, getLogger()); + final RecordReader reader = recordReaderFactory.createRecordReader(originalAttributes, rawIn, original.getSize(), getLogger()); readerSchema = reader.getSchema(); writerSchema = recordSetWriterFactory.getSchema(originalAttributes, readerSchema); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java index 57f1fbd4b276..b237a0b94b51 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java @@ -17,17 +17,6 @@ package org.apache.nifi.processors.standard; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; - import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; @@ -61,6 +50,17 @@ import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.RecordSet; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + @EventDriven @SupportsBatching @InputRequirement(Requirement.INPUT_REQUIRED) @@ -153,7 +153,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session session.read(original, new InputStreamCallback() { @Override public void process(final InputStream in) throws IOException { - try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, getLogger())) { + try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema()); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestForkRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestForkRecord.java index 562b8f04b0a0..70ec39f0fb3d 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestForkRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestForkRecord.java @@ -17,15 +17,6 @@ package org.apache.nifi.processors.standard; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.json.JsonRecordSetWriter; @@ -49,6 +40,15 @@ import org.apache.nifi.util.TestRunners; import org.junit.Test; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + public class TestForkRecord { private final String dateFormat = RecordFieldType.DATE.getDefaultFormat(); @@ -461,7 +461,8 @@ public RecordReader createRecordReader(FlowFile flowFile, InputStream in, Compon } @Override - public RecordReader createRecordReader(Map variables, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException { + public RecordReader createRecordReader(Map variables, InputStream in, long inputLength, ComponentLog logger) + throws MalformedRecordException, IOException, SchemaNotFoundException { return new JsonTreeRowRecordReader(in, logger, schema, dateFormat, timeFormat, timestampFormat); } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java index 52b7968da65c..932f6d118158 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java @@ -17,14 +17,6 @@ package org.apache.nifi.processors.standard; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.Optional; - import org.apache.nifi.avro.AvroReader; import org.apache.nifi.avro.AvroRecordSetWriter; import org.apache.nifi.csv.CSVReader; @@ -48,6 +40,14 @@ import org.junit.Before; import org.junit.Test; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Optional; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -257,9 +257,10 @@ public void testNonStrictTypeCheckWithAvroWriter() throws InitializationExceptio runner.setProperty(avroReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY); runner.enableControllerService(avroReader); final MockFlowFile validFlowFile = runner.getFlowFilesForRelationship(ValidateRecord.REL_VALID).get(0); + final byte[] validFlowFileBytes = validFlowFile.toByteArray(); try ( - final ByteArrayInputStream resultContentStream = new ByteArrayInputStream(validFlowFile.toByteArray()); - final RecordReader recordReader = avroReader.createRecordReader(validFlowFile.getAttributes(), resultContentStream, runner.getLogger()); + final ByteArrayInputStream resultContentStream = new ByteArrayInputStream(validFlowFileBytes); + final RecordReader recordReader = avroReader.createRecordReader(validFlowFile.getAttributes(), resultContentStream, validFlowFileBytes.length, runner.getLogger()); ) { final RecordSchema resultSchema = recordReader.getSchema(); assertEquals(3, resultSchema.getFieldCount()); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java index 58387efc426c..ed8fc50d2604 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java @@ -302,7 +302,7 @@ public Optional lookup(Map coordinates, Map context) throws SchemaNotFoundException, MalformedRecordException, IOException { + private Record handleResponse(InputStream is, long inputLength, Map context) throws SchemaNotFoundException, MalformedRecordException, IOException { - try (RecordReader reader = readerFactory.createRecordReader(context, is, getLogger())) { + try (RecordReader reader = readerFactory.createRecordReader(context, is, inputLength, getLogger())) { Record record = reader.nextRecord(); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java index cb42fed38e3a..c743ba6d9db3 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java @@ -37,7 +37,7 @@ public interface RecordReaderFactory extends ControllerService { /** * Create a RecordReader instance to read records from specified InputStream. - * This method calls {@link #createRecordReader(Map, InputStream, ComponentLog)} with Attributes of the specified FlowFile. + * This method calls {@link #createRecordReader(Map, InputStream, long, ComponentLog)} with Attributes of the specified FlowFile. * @param flowFile Attributes of this FlowFile are used to resolve Record Schema via Expression Language dynamically. This can be null. * * @param in InputStream containing Records. @@ -46,7 +46,7 @@ public interface RecordReaderFactory extends ControllerService { * @return Created RecordReader instance */ default RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException { - return createRecordReader(flowFile == null ? Collections.emptyMap() : flowFile.getAttributes(), in, logger); + return createRecordReader(flowFile == null ? Collections.emptyMap() : flowFile.getAttributes(), in, flowFile == null ? -1 : flowFile.getSize(), logger); } /** @@ -68,10 +68,11 @@ default RecordReader createRecordReader(FlowFile flowFile, InputStream in, Compo * @param variables A map containing variables which is used to resolve the Record Schema dynamically via Expression Language. * This can be null or empty. * @param in InputStream containing Records. + * @param inputLength the length of the content to read from the InputStream in bytes, a negative value indicates an unknown or unbound size * @param logger A logger bound to a component * * @return Created RecordReader instance */ - RecordReader createRecordReader(Map variables, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException; + RecordReader createRecordReader(Map variables, InputStream in, long inputLength, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException; } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java index 11379b97dfe4..aae5b5961d5a 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java @@ -94,7 +94,7 @@ protected SchemaAccessStrategy getSchemaAccessStrategy(String strategy, SchemaRe } @Override - public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) throws IOException, SchemaNotFoundException { final String schemaAccessStrategy = getConfigurationContext().getProperty(getSchemaAcessStrategyDescriptor()).getValue(); if (EMBEDDED_AVRO_SCHEMA.getValue().equals(schemaAccessStrategy)) { return new AvroReaderWithEmbeddedSchema(in); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java index b1a840a6cb99..b71971165df4 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java @@ -139,7 +139,7 @@ public void storeStaticProperties(final ConfigurationContext context) { } @Override - public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) throws IOException, SchemaNotFoundException { // Use Mark/Reset of a BufferedInputStream in case we read from the Input Stream for the header. in.mark(1024 * 1024); final RecordSchema schema = getSchema(variables, new NonCloseableInputStream(in), null); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java index 099da2a102f5..ff2421162187 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java @@ -267,7 +267,7 @@ public Set getSuppliedSchemaFields() { } @Override - public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) throws IOException, SchemaNotFoundException { final RecordSchema schema = getSchema(variables, in, null); return new GrokRecordReader(in, grok, schema, recordSchemaFromGrok, appendUnmatchedLine); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java index a59bc19325c1..f16106d61e09 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java @@ -162,7 +162,8 @@ protected AllowableValue getDefaultSchemaAccessStrategy() { } @Override - public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException { + public RecordReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) + throws IOException, MalformedRecordException, SchemaNotFoundException { final InputStream bufferedIn = new BufferedInputStream(in); final RecordSchema schema = getSchema(variables, bufferedIn, null); return new JsonPathRowRecordReader(jsonPaths, schema, bufferedIn, logger, dateFormat, timeFormat, timestampFormat); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java index c80542120d01..d00799c834fd 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java @@ -105,7 +105,8 @@ protected AllowableValue getDefaultSchemaAccessStrategy() { } @Override - public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException { + public RecordReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) + throws IOException, MalformedRecordException, SchemaNotFoundException { final RecordSchema schema = getSchema(variables, in, null); return new JsonTreeRowRecordReader(in, logger, schema, dateFormat, timeFormat, timestampFormat); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/Syslog5424Reader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/Syslog5424Reader.java index 7f29db2d2b4e..89061eb608d7 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/Syslog5424Reader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/Syslog5424Reader.java @@ -143,7 +143,7 @@ public Set getSuppliedSchemaFields() { } @Override - public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) throws IOException, SchemaNotFoundException { final RecordSchema schema = getSchema(variables, in, null); return new Syslog5424RecordReader(parser, in, schema); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/SyslogReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/SyslogReader.java index 9b7eda93c030..2dd2b03926cc 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/SyslogReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/SyslogReader.java @@ -136,7 +136,7 @@ public Set getSuppliedSchemaFields() { } @Override - public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException { + public RecordReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) throws IOException, SchemaNotFoundException { final RecordSchema schema = getSchema(variables, in, null); return new SyslogRecordReader(parser, in, schema); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java index d96d397f917e..2f800c390fe5 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java @@ -160,7 +160,8 @@ protected AllowableValue getDefaultSchemaAccessStrategy() { } @Override - public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException, MalformedRecordException { + public RecordReader createRecordReader(final Map variables, final InputStream in, final long inputLength, final ComponentLog logger) + throws IOException, SchemaNotFoundException, MalformedRecordException { final ConfigurationContext context = getConfigurationContext(); final RecordSchema schema = getSchema(variables, in, null); From 6731e4c8eeee69e518362839e470274c90b58a5b Mon Sep 17 00:00:00 2001 From: Bryan Bende Date: Wed, 4 Sep 2019 11:49:19 -0400 Subject: [PATCH 10/11] NIFI-6089 Improve javadoc for newly added inputLength argument in RecordReaderFactory --- .../org/apache/nifi/serialization/RecordReaderFactory.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java index c743ba6d9db3..f86541ac85ed 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java @@ -68,7 +68,9 @@ default RecordReader createRecordReader(FlowFile flowFile, InputStream in, Compo * @param variables A map containing variables which is used to resolve the Record Schema dynamically via Expression Language. * This can be null or empty. * @param in InputStream containing Records. - * @param inputLength the length of the content to read from the InputStream in bytes, a negative value indicates an unknown or unbound size + * @param inputLength A hint about the length of the content to read from the InputStream. This value is specified in bytes and a negative value indicates + * an unknown or unbound size. Some reader implementations must be able to seek to the end of the InputStream and will require knowing the length. + * Other implementations may be ignore this hint and may read the entire InputStream. * @param logger A logger bound to a component * * @return Created RecordReader instance From 78d9f13ded17901741ffe72801611c4a59f0489a Mon Sep 17 00:00:00 2001 From: Brandon Date: Fri, 16 Aug 2019 13:24:13 +0000 Subject: [PATCH 11/11] NIFI-6416: This closes #3654. Removing swap locations on SWAP_IN Signed-off-by: Joe Witt --- .../controller/repository/WriteAheadFlowFileRepository.java | 2 +- .../repository/TestWriteAheadFlowFileRepository.java | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java index 133751179dbd..779c29c421b0 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java @@ -515,7 +515,7 @@ public void swapFlowFilesIn(final String swapLocation, final List