You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by da...@apache.org on 2015/10/13 02:50:51 UTC

[01/11] nifi git commit: [NIFI-774] Create a DeleteS3Object Processor

Repository: nifi
Updated Branches:
  refs/heads/master f6d342795 -> 49ee06b0a


[NIFI-774] Create a DeleteS3Object Processor


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/d1dbd376
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/d1dbd376
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/d1dbd376

Branch: refs/heads/master
Commit: d1dbd37629baeb9a1570f47f7583d07f54b38b5c
Parents: e4e263c
Author: Yu ISHIKAWA <yu...@gmail.com>
Authored: Thu Jul 23 13:13:15 2015 +0900
Committer: Yuu ISHIKAWA <yu...@gmail.com>
Committed: Tue Sep 1 22:35:11 2015 +0900

----------------------------------------------------------------------
 .../nifi/processors/aws/s3/DeleteS3Object.java  | 108 +++++++++++++++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../processors/aws/s3/TestDeleteS3Object.java   | 137 +++++++++++++++++++
 3 files changed, 246 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/d1dbd376/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
new file mode 100644
index 0000000..3be7a15
--- /dev/null
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
@@ -0,0 +1,108 @@
+/*
+ * 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.processors.aws.s3;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.DeleteObjectRequest;
+import com.amazonaws.services.s3.model.DeleteVersionRequest;
+
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+
+
+@SupportsBatching
+@SeeAlso({PutS3Object.class})
+@Tags({"Amazon", "S3", "AWS", "Archive", "Delete"})
+@CapabilityDescription("Deletes FlowFiles on an Amazon S3 Bucket. " +
+        "And the FlowFiles are checked if exists or not before deleting.")
+public class DeleteS3Object extends AbstractS3Processor {
+
+    public static final PropertyDescriptor VERSION_ID = new PropertyDescriptor.Builder()
+            .name("Version")
+            .description("The Version of the Object to delete")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(true)
+            .required(false)
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(KEY, BUCKET, ACCESS_KEY, SECRET_KEY, CREDENTAILS_FILE, REGION, TIMEOUT, VERSION_ID,
+                    FULL_CONTROL_USER_LIST, READ_USER_LIST, WRITE_USER_LIST, READ_ACL_LIST, WRITE_ACL_LIST, OWNER));
+
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                .expressionLanguageSupported(true)
+                .dynamic(true)
+                .build();
+    }
+
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final long startNanos = System.nanoTime();
+
+        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue();
+        final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
+        final String versionId = context.getProperty(VERSION_ID).evaluateAttributeExpressions(flowFile).getValue();
+
+        final AmazonS3 s3 = getClient();
+        try {
+            // Checks if the key exists or not
+            // If there is no such a key, then throws a exception
+            s3.getObjectMetadata(bucket, key);
+
+            // Deletes a key on Amazon S3
+            if (versionId == null) {
+                final DeleteObjectRequest r = new DeleteObjectRequest(bucket, key);
+                s3.deleteObject(r);
+            } else {
+                final DeleteVersionRequest r = new DeleteVersionRequest(bucket, key, versionId);
+                s3.deleteVersion(r);
+            }
+        } catch (final AmazonServiceException ase) {
+            getLogger().error("Failed to delete S3 Object for {}; routing to failure", new Object[]{flowFile, ase});
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        session.transfer(flowFile, REL_SUCCESS);
+        final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+        getLogger().info("Successfully delete S3 Object for {} in {} millis; routing to success", new Object[]{flowFile, transferMillis});
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1dbd376/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index 4f2405c..d0d1e73 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -14,6 +14,7 @@
 # limitations under the License.
 org.apache.nifi.processors.aws.s3.FetchS3Object
 org.apache.nifi.processors.aws.s3.PutS3Object
+org.apache.nifi.processors.aws.s3.DeleteS3Object
 org.apache.nifi.processors.aws.sns.PutSNS
 org.apache.nifi.processors.aws.sqs.GetSQS
 org.apache.nifi.processors.aws.sqs.PutSQS

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1dbd376/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
new file mode 100644
index 0000000..dfe6edb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
@@ -0,0 +1,137 @@
+/*
+ * 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.processors.aws.s3;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.amazonaws.auth.PropertiesCredentials;
+import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.model.*;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+
+@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created")
+public class TestDeleteS3Object {
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
+
+    // When you want to test this, you should create a bucket on Amazon S3 as follows.
+    private static final String TEST_REGION = "ap-northeast-1";
+    private static final String TEST_BUCKET = "test-bucket-00000000-0000-0000-0000-1234567890123";
+
+    @BeforeClass
+    public static void oneTimeSetUp() {
+        // Creates a new bucket for this test
+        try {
+            PropertiesCredentials credentials = new PropertiesCredentials(new FileInputStream(CREDENTIALS_FILE));
+            AmazonS3Client client = new AmazonS3Client(credentials);
+            CreateBucketRequest request = new CreateBucketRequest(TEST_BUCKET, TEST_REGION);
+            client.createBucket(request);
+        } catch (final AmazonS3Exception e) {
+            System.out.println(TEST_BUCKET + " already exists.");
+        } catch (final IOException e) {
+            System.out.println(CREDENTIALS_FILE + " doesn't exist.");
+        }
+    }
+
+    @AfterClass
+    public static void oneTimeTearDown() throws IOException {
+        // Delete a bucket for this test
+        PropertiesCredentials credentials = new PropertiesCredentials(new FileInputStream(CREDENTIALS_FILE));
+        AmazonS3Client client = new AmazonS3Client(credentials);
+        DeleteBucketRequest dbr = new DeleteBucketRequest(TEST_BUCKET);
+        client.deleteBucket(dbr);
+    }
+
+    @Test
+    public void testSimpleDelete() throws IOException {
+        // Prepares for this test
+        uploadTestFile("hello.txt");
+
+        DeleteS3Object deleter = new DeleteS3Object();
+        final TestRunner runner = TestRunners.newTestRunner(deleter);
+        runner.setProperty(DeleteS3Object.CREDENTAILS_FILE, CREDENTIALS_FILE);
+        runner.setProperty(DeleteS3Object.REGION, TEST_REGION);
+        runner.setProperty(DeleteS3Object.BUCKET, TEST_BUCKET);
+        runner.setProperty(DeleteS3Object.KEY, "hello.txt");
+
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+        runner.run(1);
+
+        runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_SUCCESS, 1);
+    }
+
+    @Test
+    public void testDeleteFolder() throws IOException {
+        // Prepares for this test
+        uploadTestFile("folder/1.txt");
+
+        DeleteS3Object deleter = new DeleteS3Object();
+        final TestRunner runner = TestRunners.newTestRunner(deleter);
+        runner.setProperty(DeleteS3Object.CREDENTAILS_FILE, CREDENTIALS_FILE);
+        runner.setProperty(DeleteS3Object.REGION, TEST_REGION);
+        runner.setProperty(DeleteS3Object.BUCKET, TEST_BUCKET);
+        runner.setProperty(DeleteS3Object.KEY, "folder/1.txt");
+
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+        runner.run(1);
+
+        runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_SUCCESS, 1);
+    }
+
+    @Test
+    public void testTryToDeleteNotExistingFile() throws IOException {
+        DeleteS3Object deleter = new DeleteS3Object();
+        final TestRunner runner = TestRunners.newTestRunner(deleter);
+        runner.setProperty(DeleteS3Object.CREDENTAILS_FILE, CREDENTIALS_FILE);
+        runner.setProperty(DeleteS3Object.REGION, TEST_REGION);
+        runner.setProperty(DeleteS3Object.BUCKET, TEST_BUCKET);
+        runner.setProperty(DeleteS3Object.BUCKET, "no-such-a-key");
+
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "no-such-a-file");
+        runner.enqueue(new byte[0], attrs);
+        runner.run(1);
+
+        runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_FAILURE, 1);
+    }
+
+    // Uploads a test file
+    private void uploadTestFile(String key) throws IOException {
+        PropertiesCredentials credentials = new PropertiesCredentials(new FileInputStream(CREDENTIALS_FILE));
+        AmazonS3Client client = new AmazonS3Client(credentials);
+        URL fileURL = this.getClass().getClassLoader().getResource("hello.txt");
+        File file = new File(fileURL.getPath());
+        PutObjectRequest putRequest = new PutObjectRequest(TEST_BUCKET, key, file);
+        PutObjectResult result = client.putObject(putRequest);
+    }
+}


[02/11] nifi git commit: Ignore the test suite

Posted by da...@apache.org.
Ignore the test suite


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/7e683493
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/7e683493
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/7e683493

Branch: refs/heads/master
Commit: 7e6834937b01f035666b113a6c2735e4156ea4f3
Parents: d1dbd37
Author: Yuu ISHIKAWA <yu...@gmail.com>
Authored: Tue Sep 1 22:36:50 2015 +0900
Committer: Yuu ISHIKAWA <yu...@gmail.com>
Committed: Tue Sep 1 22:36:50 2015 +0900

----------------------------------------------------------------------
 .../org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/7e683493/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
index dfe6edb..cac55e5 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
@@ -25,17 +25,17 @@ import java.util.Map;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import com.amazonaws.auth.PropertiesCredentials;
 import com.amazonaws.services.s3.AmazonS3Client;
 import com.amazonaws.services.s3.model.*;
+
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 
 
-@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created")
+//@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created")
 public class TestDeleteS3Object {
 
     private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";


[04/11] nifi git commit: Add @Override annotations

Posted by da...@apache.org.
Add @Override annotations


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/213f507f
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/213f507f
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/213f507f

Branch: refs/heads/master
Commit: 213f507f53b3afa3eef3398ebdd3006fa1cbe292
Parents: 6cbc6db
Author: Yu ISHIKAWA <yu...@gmail.com>
Authored: Wed Sep 2 13:02:39 2015 +0900
Committer: Yu ISHIKAWA <yu...@gmail.com>
Committed: Wed Sep 2 13:02:39 2015 +0900

----------------------------------------------------------------------
 .../java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java     | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/213f507f/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
index 85fc70e..2cc00db 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
@@ -56,10 +56,12 @@ public class DeleteS3Object extends AbstractS3Processor {
             Arrays.asList(KEY, BUCKET, ACCESS_KEY, SECRET_KEY, CREDENTAILS_FILE, REGION, TIMEOUT, VERSION_ID,
                     FULL_CONTROL_USER_LIST, READ_USER_LIST, WRITE_USER_LIST, READ_ACL_LIST, WRITE_ACL_LIST, OWNER));
 
+    @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         return properties;
     }
 
+    @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
         FlowFile flowFile = session.get();
         if (flowFile == null) {


[09/11] nifi git commit: Remove `REL_NOT_FOUND`

Posted by da...@apache.org.
Remove `REL_NOT_FOUND`


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/eb1d6b55
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/eb1d6b55
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/eb1d6b55

Branch: refs/heads/master
Commit: eb1d6b554cc589d7a62b4bf452bf4ebfbad1f8ff
Parents: f718b4b
Author: Yu ISHIKAWA <yu...@gmail.com>
Authored: Fri Sep 11 18:05:20 2015 +0900
Committer: Yu ISHIKAWA <yu...@gmail.com>
Committed: Fri Sep 11 18:05:20 2015 +0900

----------------------------------------------------------------------
 .../org/apache/nifi/processors/aws/s3/DeleteS3Object.java | 10 +---------
 1 file changed, 1 insertion(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/eb1d6b55/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
index c8950c3..836e0d8 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
@@ -18,15 +18,14 @@ package org.apache.nifi.processors.aws.s3;
 
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.DeleteObjectRequest;
 import com.amazonaws.services.s3.model.DeleteVersionRequest;
+
 import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
@@ -35,7 +34,6 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.util.StandardValidators;
 
 
@@ -46,9 +44,6 @@ import org.apache.nifi.processor.util.StandardValidators;
         "And the FlowFiles are checked if exists or not before deleting.")
 public class DeleteS3Object extends AbstractS3Processor {
 
-    public static final Relationship REL_NOT_FOUND = new Relationship.Builder().name("not found")
-            .description("FlowFiles are routed to 'not found' if it doesn't exist on Amazon S3").build();
-
     public static final PropertyDescriptor VERSION_ID = new PropertyDescriptor.Builder()
             .name("Version")
             .description("The Version of the Object to delete")
@@ -61,9 +56,6 @@ public class DeleteS3Object extends AbstractS3Processor {
             Arrays.asList(KEY, BUCKET, ACCESS_KEY, SECRET_KEY, CREDENTAILS_FILE, REGION, TIMEOUT, VERSION_ID,
                     FULL_CONTROL_USER_LIST, READ_USER_LIST, WRITE_USER_LIST, READ_ACL_LIST, WRITE_ACL_LIST, OWNER));
 
-    public static final Set<Relationship> relationships = Collections.unmodifiableSet(
-            new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE, REL_NOT_FOUND)));
-
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         return properties;


[06/11] nifi git commit: Fix a type and modify an error message

Posted by da...@apache.org.
Fix a type and modify an error message


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c492a1aa
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c492a1aa
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c492a1aa

Branch: refs/heads/master
Commit: c492a1aaae14efae3506ae43979fb93ba6655438
Parents: 0334f04
Author: Yuu ISHIKAWA <yu...@gmail.com>
Authored: Wed Sep 2 21:54:02 2015 +0900
Committer: Yuu ISHIKAWA <yu...@gmail.com>
Committed: Wed Sep 2 21:54:02 2015 +0900

----------------------------------------------------------------------
 .../java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/c492a1aa/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
index 803a6ab..dffcab8 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
@@ -94,7 +94,7 @@ public class DeleteS3Object extends AbstractS3Processor {
         try {
           s3.getObjectMetadata(bucket, key);
         } catch (final AmazonServiceException ase) {
-            getLogger().error("Not found sucha a file and folder on Amazon S3 {}", new Object[]{flowFile, ase});
+            getLogger().error("Not found such a S3 object for {}; routing to not found", new Object[]{flowFile, ase});
             session.transfer(flowFile, REL_NOT_FOUND);
             return;
         }


[08/11] nifi git commit: Modify an error message

Posted by da...@apache.org.
Modify an error message


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/f718b4bf
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/f718b4bf
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/f718b4bf

Branch: refs/heads/master
Commit: f718b4bf46f441fde2f02065b6d55ac294fe60ce
Parents: d32a32a
Author: Yu ISHIKAWA <yu...@gmail.com>
Authored: Mon Sep 7 23:34:22 2015 +0900
Committer: Yu ISHIKAWA <yu...@gmail.com>
Committed: Mon Sep 7 23:34:22 2015 +0900

----------------------------------------------------------------------
 .../java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/f718b4bf/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
index 082a80d..603b06a 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
@@ -56,7 +56,7 @@ public class TestDeleteS3Object {
             CreateBucketRequest request = new CreateBucketRequest(TEST_BUCKET, TEST_REGION);
             client.createBucket(request);
         } catch (final AmazonS3Exception e) {
-            System.out.println(TEST_BUCKET + " already exists.");
+            System.out.println("Can't create the key " + TEST_BUCKET + ":" + e.toString());
         } catch (final IOException e) {
             System.out.println(CREDENTIALS_FILE + " doesn't exist.");
         }


[10/11] nifi git commit: Merge remote-tracking branch 'yu/NIFI-774'

Posted by da...@apache.org.
Merge remote-tracking branch 'yu/NIFI-774'


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/fad81d87
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/fad81d87
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/fad81d87

Branch: refs/heads/master
Commit: fad81d872d51d797bcc681977cfcb5fbda482257
Parents: f6d3427 eb1d6b5
Author: danbress <db...@onyxconsults.com>
Authored: Mon Oct 12 09:55:52 2015 -0400
Committer: danbress <db...@onyxconsults.com>
Committed: Mon Oct 12 09:55:52 2015 -0400

----------------------------------------------------------------------
 .../processors/aws/AbstractAWSProcessor.java    |   2 +-
 .../nifi/processors/aws/s3/DeleteS3Object.java  |  98 +++++++++++++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../processors/aws/s3/TestDeleteS3Object.java   | 140 +++++++++++++++++++
 4 files changed, 240 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[05/11] nifi git commit: Add a relation for "not found" and that is transfered if a target key doesn't exist on S3

Posted by da...@apache.org.
Add a relation for "not found" and that is transfered if a target key doesn't exist on S3


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/0334f046
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/0334f046
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/0334f046

Branch: refs/heads/master
Commit: 0334f04640c7ae0b070214df7e356b2b05a8051c
Parents: 213f507
Author: Yu ISHIKAWA <yu...@gmail.com>
Authored: Wed Sep 2 13:03:05 2015 +0900
Committer: Yu ISHIKAWA <yu...@gmail.com>
Committed: Wed Sep 2 13:48:39 2015 +0900

----------------------------------------------------------------------
 .../processors/aws/AbstractAWSProcessor.java    |  2 +-
 .../nifi/processors/aws/s3/DeleteS3Object.java  | 30 ++++++++++++++++----
 .../processors/aws/s3/TestDeleteS3Object.java   | 19 +++++++++++--
 3 files changed, 42 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/0334f046/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
index a781ff9..e2ae31e 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
@@ -54,7 +54,7 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
     public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
             .description("FlowFiles are routed to failure if unable to be copied to Amazon S3").build();
 
-    public static final Set<Relationship> relationships = Collections.unmodifiableSet(
+    public static Set<Relationship> relationships = Collections.unmodifiableSet(
             new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE)));
 
     public static final PropertyDescriptor CREDENTAILS_FILE = new PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/0334f046/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
index 2cc00db..803a6ab 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
@@ -18,14 +18,15 @@ package org.apache.nifi.processors.aws.s3;
 
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.DeleteObjectRequest;
 import com.amazonaws.services.s3.model.DeleteVersionRequest;
-
 import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
@@ -34,6 +35,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.util.StandardValidators;
 
 
@@ -44,6 +46,9 @@ import org.apache.nifi.processor.util.StandardValidators;
         "And the FlowFiles are checked if exists or not before deleting.")
 public class DeleteS3Object extends AbstractS3Processor {
 
+    public static final Relationship REL_NOT_FOUND = new Relationship.Builder().name("not found")
+            .description("FlowFiles are routed to 'not found' if it doesn't exist on Amazon S3").build();
+
     public static final PropertyDescriptor VERSION_ID = new PropertyDescriptor.Builder()
             .name("Version")
             .description("The Version of the Object to delete")
@@ -56,6 +61,14 @@ public class DeleteS3Object extends AbstractS3Processor {
             Arrays.asList(KEY, BUCKET, ACCESS_KEY, SECRET_KEY, CREDENTAILS_FILE, REGION, TIMEOUT, VERSION_ID,
                     FULL_CONTROL_USER_LIST, READ_USER_LIST, WRITE_USER_LIST, READ_ACL_LIST, WRITE_ACL_LIST, OWNER));
 
+    public static final Set<Relationship> relationships = Collections.unmodifiableSet(
+            new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE, REL_NOT_FOUND)));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         return properties;
@@ -75,12 +88,19 @@ public class DeleteS3Object extends AbstractS3Processor {
         final String versionId = context.getProperty(VERSION_ID).evaluateAttributeExpressions(flowFile).getValue();
 
         final AmazonS3 s3 = getClient();
+
+        // Checks if the key exists or not
+        // If there is no such a key, then throws a exception
         try {
-            // Checks if the key exists or not
-            // If there is no such a key, then throws a exception
-            s3.getObjectMetadata(bucket, key);
+          s3.getObjectMetadata(bucket, key);
+        } catch (final AmazonServiceException ase) {
+            getLogger().error("Not found sucha a file and folder on Amazon S3 {}", new Object[]{flowFile, ase});
+            session.transfer(flowFile, REL_NOT_FOUND);
+            return;
+        }
 
-            // Deletes a key on Amazon S3
+        // Deletes a key on Amazon S3
+        try {
             if (versionId == null) {
                 final DeleteObjectRequest r = new DeleteObjectRequest(bucket, key);
                 s3.deleteObject(r);

http://git-wip-us.apache.org/repos/asf/nifi/blob/0334f046/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
index cac55e5..04d9e61 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
@@ -22,15 +22,21 @@ import java.io.IOException;
 import java.net.URL;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 import com.amazonaws.auth.PropertiesCredentials;
 import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.model.*;
-
+import com.amazonaws.services.s3.model.AmazonS3Exception;
+import com.amazonaws.services.s3.model.CreateBucketRequest;
+import com.amazonaws.services.s3.model.DeleteBucketRequest;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
+import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 
@@ -122,7 +128,14 @@ public class TestDeleteS3Object {
         runner.enqueue(new byte[0], attrs);
         runner.run(1);
 
-        runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_FAILURE, 1);
+        runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_NOT_FOUND, 1);
+    }
+
+    @Test
+    public void testGetRelationships() {
+        DeleteS3Object deleter = new DeleteS3Object();
+        Set<Relationship> relationships = deleter.getRelationships();
+        assertEquals(relationships.size(), 3);
     }
 
     // Uploads a test file


[03/11] nifi git commit: Remove `getSupportedDynamicPropertyDescriptor`

Posted by da...@apache.org.
Remove `getSupportedDynamicPropertyDescriptor`


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6cbc6db8
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6cbc6db8
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6cbc6db8

Branch: refs/heads/master
Commit: 6cbc6db860d730954ce8cadc56b6d8446fe085d8
Parents: 7e68349
Author: Yuu ISHIKAWA <yu...@gmail.com>
Authored: Wed Sep 2 06:49:46 2015 +0900
Committer: Yuu ISHIKAWA <yu...@gmail.com>
Committed: Wed Sep 2 06:49:46 2015 +0900

----------------------------------------------------------------------
 .../org/apache/nifi/processors/aws/s3/DeleteS3Object.java   | 9 ---------
 1 file changed, 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/6cbc6db8/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
index 3be7a15..85fc70e 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
@@ -60,15 +60,6 @@ public class DeleteS3Object extends AbstractS3Processor {
         return properties;
     }
 
-    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder()
-                .name(propertyDescriptorName)
-                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(true)
-                .dynamic(true)
-                .build();
-    }
-
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
         FlowFile flowFile = session.get();
         if (flowFile == null) {


[11/11] nifi git commit: NIFI-774 Ignore integration style test

Posted by da...@apache.org.
NIFI-774 Ignore integration style test


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/49ee06b0
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/49ee06b0
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/49ee06b0

Branch: refs/heads/master
Commit: 49ee06b0a256b9b4b7550acccb6facceb3c31c21
Parents: fad81d8
Author: danbress <db...@onyxconsults.com>
Authored: Mon Oct 12 19:59:46 2015 -0400
Committer: danbress <db...@onyxconsults.com>
Committed: Mon Oct 12 19:59:46 2015 -0400

----------------------------------------------------------------------
 .../org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/49ee06b0/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
index 603b06a..509a31f 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.Map;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import com.amazonaws.auth.PropertiesCredentials;
@@ -38,7 +39,7 @@ import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 
 
-//@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created")
+@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created")
 public class TestDeleteS3Object {
 
     private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";


[07/11] nifi git commit: Remove the process to check if the target key exists or not

Posted by da...@apache.org.
Remove the process to check if the target key exists or not


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/d32a32a9
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/d32a32a9
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/d32a32a9

Branch: refs/heads/master
Commit: d32a32a92d29707d22899862594a2679d3fb2273
Parents: c492a1a
Author: Yu ISHIKAWA <yu...@gmail.com>
Authored: Mon Sep 7 23:32:48 2015 +0900
Committer: Yu ISHIKAWA <yu...@gmail.com>
Committed: Mon Sep 7 23:32:48 2015 +0900

----------------------------------------------------------------------
 .../nifi/processors/aws/s3/DeleteS3Object.java       | 15 ---------------
 .../nifi/processors/aws/s3/TestDeleteS3Object.java   | 14 ++------------
 2 files changed, 2 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/d32a32a9/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
index dffcab8..c8950c3 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
@@ -65,11 +65,6 @@ public class DeleteS3Object extends AbstractS3Processor {
             new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE, REL_NOT_FOUND)));
 
     @Override
-    public Set<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         return properties;
     }
@@ -89,16 +84,6 @@ public class DeleteS3Object extends AbstractS3Processor {
 
         final AmazonS3 s3 = getClient();
 
-        // Checks if the key exists or not
-        // If there is no such a key, then throws a exception
-        try {
-          s3.getObjectMetadata(bucket, key);
-        } catch (final AmazonServiceException ase) {
-            getLogger().error("Not found such a S3 object for {}; routing to not found", new Object[]{flowFile, ase});
-            session.transfer(flowFile, REL_NOT_FOUND);
-            return;
-        }
-
         // Deletes a key on Amazon S3
         try {
             if (versionId == null) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/d32a32a9/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
index 04d9e61..082a80d 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestDeleteS3Object.java
@@ -22,12 +22,9 @@ import java.io.IOException;
 import java.net.URL;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Set;
-
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
 
 import com.amazonaws.auth.PropertiesCredentials;
 import com.amazonaws.services.s3.AmazonS3Client;
@@ -36,7 +33,7 @@ import com.amazonaws.services.s3.model.CreateBucketRequest;
 import com.amazonaws.services.s3.model.DeleteBucketRequest;
 import com.amazonaws.services.s3.model.PutObjectRequest;
 import com.amazonaws.services.s3.model.PutObjectResult;
-import org.apache.nifi.processor.Relationship;
+
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 
@@ -128,14 +125,7 @@ public class TestDeleteS3Object {
         runner.enqueue(new byte[0], attrs);
         runner.run(1);
 
-        runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_NOT_FOUND, 1);
-    }
-
-    @Test
-    public void testGetRelationships() {
-        DeleteS3Object deleter = new DeleteS3Object();
-        Set<Relationship> relationships = deleter.getRelationships();
-        assertEquals(relationships.size(), 3);
+        runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_FAILURE, 1);
     }
 
     // Uploads a test file