turcsanyip commented on code in PR #6540:
URL: https://github.com/apache/nifi/pull/6540#discussion_r1026823892
##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/PutAzureDataLakeStorage.java:
##########
@@ -220,18 +213,25 @@ static void uploadContent(DataLakeFileClient fileClient,
InputStream in, long le
}
//Visible for testing
- DataLakeFileClient renameFile(final String fileName, final String
directoryPath, final DataLakeFileClient fileClient, final boolean overwrite) {
+ String renameFile(final DataLakeFileClient sourceFileClient, final String
destinationDirectory, final String destinationFileName, final String
conflictResolution) {
try {
final DataLakeRequestConditions destinationCondition = new
DataLakeRequestConditions();
- if (!overwrite) {
+ if (!conflictResolution.equals(REPLACE_RESOLUTION)) {
destinationCondition.setIfNoneMatch("*");
}
- final String destinationPath = createPath(directoryPath, fileName);
- return fileClient.renameWithResponse(null, destinationPath, null,
destinationCondition, null, null).getValue();
+ final String destinationPath = createPath(destinationDirectory,
destinationFileName);
+ return sourceFileClient.renameWithResponse(null, destinationPath,
null, destinationCondition, null, null).getValue().getFileUrl();
} catch (DataLakeStorageException dataLakeStorageException) {
- getLogger().error("Renaming File [{}] failed",
fileClient.getFileName(), dataLakeStorageException);
- removeTempFile(fileClient);
- throw dataLakeStorageException;
+ removeTempFile(sourceFileClient);
+ if (dataLakeStorageException.getStatusCode() == 409 &&
conflictResolution.equals(IGNORE_RESOLUTION)) {
+ getLogger().info("File with the same name [{}] already exists.
Remote file not modified due to {} being set to '{}'.",
+ sourceFileClient.getFileName(),
CONFLICT_RESOLUTION.getDisplayName(), conflictResolution);
+ return null;
+ } else {
+ // FAIL_RESOLUTION
+ getLogger().error("Renaming File [{}] failed",
sourceFileClient.getFileName(), dataLakeStorageException);
Review Comment:
The error is logged (and therefore appears on the bulletin) twice. It would
be enough to log the error in "main" catch block in `onTrigger()`.
##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITPutAzureDataLakeStorage.java:
##########
@@ -284,17 +285,18 @@ public void testPutFileButFailedToRename() {
final String componentId = "componentId";
final DataLakeFileClient fileClient = mock(DataLakeFileClient.class);
final Response<DataLakeFileClient> response = mock(Response.class);
+ final DataLakeStorageException exception =
mock(DataLakeStorageException.class);
//Mock logger
when(initContext.getIdentifier()).thenReturn(componentId);
MockComponentLog componentLog = new MockComponentLog(componentId,
processor);
when(initContext.getLogger()).thenReturn(componentLog);
processor.initialize(initContext);
//Mock renameWithResponse Azure method
when(fileClient.renameWithResponse(isNull(), anyString(), isNull(),
any(DataLakeRequestConditions.class), isNull(), isNull())).thenReturn(response);
- when(response.getValue()).thenThrow(DataLakeStorageException.class);
when(fileClient.getFileName()).thenReturn(FILE_NAME);
-
- assertThrows(DataLakeStorageException.class, () ->
processor.renameFile(FILE_NAME, "", fileClient, false));
+ when(exception.getStatusCode()).thenReturn(405);
+ when(response.getValue()).thenThrow(exception);
+ assertThrows(DataLakeStorageException.class, () ->
processor.renameFile(fileClient, "", FILE_NAME, FAIL_RESOLUTION));
Review Comment:
If I understand correctly, here we test the "other" error case (non 409 /
non-conflicting). Generic errors should be handled the same way in all conflict
resolution strategies so the test should be extended with those too:
```suggestion
when(exception.getStatusCode()).thenReturn(405);
when(response.getValue()).thenThrow(exception);
assertThrows(DataLakeStorageException.class, () ->
processor.renameFile(fileClient, "", FILE_NAME, FAIL_RESOLUTION));
assertThrows(DataLakeStorageException.class, () ->
processor.renameFile(fileClient, "", FILE_NAME, REPLACE_RESOLUTION));
assertThrows(DataLakeStorageException.class, () ->
processor.renameFile(fileClient, "", FILE_NAME, IGNORE_RESOLUTION));
verify(fileClient, times(3)).delete();
```
This and the previous one are unit tests. Not clear why they were added in
the IT test class originally. It may be worth moving them to a separate unit
test.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]