Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/238#discussion_r53674055
  
    --- Diff: 
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java
 ---
    @@ -0,0 +1,230 @@
    +/*
    + * 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 com.amazonaws.services.s3.AmazonS3;
    +import com.amazonaws.services.s3.model.ListObjectsRequest;
    +import com.amazonaws.services.s3.model.ObjectListing;
    +import com.amazonaws.services.s3.model.S3ObjectSummary;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +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.components.state.Scope;
    +import org.apache.nifi.components.state.StateMap;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.flowfile.attributes.CoreAttributes;
    +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;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +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.concurrent.TimeUnit;
    +
    +@TriggerSerially
    +@TriggerWhenEmpty
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@Tags({"Amazon", "S3", "AWS", "list"})
    +@CapabilityDescription("Retrieves a listing of objects from an S3 bucket. 
For each object that is listed, creates a FlowFile that represents \"\n" +
    +        "        + \"the object so that it can be fetched in conjunction 
with FetchS3Object. This Processor is designed to run on Primary Node only 
\"\n" +
    +        "        + \"in a cluster. If the primary node changes, the new 
Primary Node will pick up where the previous node left off without duplicating 
\"\n" +
    +        "        + \"all of the data.")
    +@Stateful(scopes = Scope.CLUSTER, description = "After performing a 
listing of keys, the timestamp of the newest key is stored, "
    +        + "along with the keys that share that same timestamp. This allows 
the Processor to list only keys that have been added or modified after "
    +        + "this date the next time that the Processor is run. State is 
stored across the cluster so that this Processor can be run on Primary Node 
only and if a new Primary "
    +        + "Node is selected, the new node can pick up where the previous 
node left off, without duplicating the data.")
    +@WritesAttributes({
    +        @WritesAttribute(attribute = "s3.bucket", description = "The name 
of the S3 bucket"),
    +        @WritesAttribute(attribute = "filename", description = "The name 
of the file"),
    +        @WritesAttribute(attribute = "s3.etag", description = "The ETag 
that can be used to see if the file has changed"),
    +        @WritesAttribute(attribute = "s3.lastModified", description = "The 
last modified time in milliseconds since epoch in UTC time"),
    +        @WritesAttribute(attribute = "s3.length", description = "The size 
of the object in bytes"),
    +        @WritesAttribute(attribute = "s3.storageClass", description = "The 
storage class of the object"),})
    +@SeeAlso({FetchS3Object.class, PutS3Object.class, DeleteS3Object.class})
    +public class ListS3 extends AbstractS3Processor {
    +
    +    public static final PropertyDescriptor DELIMITER = new 
PropertyDescriptor.Builder()
    +            .name("Delimiter")
    +            .expressionLanguageSupported(false)
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .description("The string used to delimit directories within 
the bucket. Please consult the AWS documentation " +
    +                    "for the correct use of this field.")
    +            .build();
    +
    +    public static final PropertyDescriptor PREFIX = new 
PropertyDescriptor.Builder()
    +            .name("Prefix")
    +            .expressionLanguageSupported(false)
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .description("The prefix used to filter the object list. In 
most cases, it should end with a forward slash ('/').")
    +            .build();
    +
    +    public static final List<PropertyDescriptor> properties = 
Collections.unmodifiableList(
    +            Arrays.asList(BUCKET, KEY, REGION, ACCESS_KEY, SECRET_KEY, 
CREDENTIALS_FILE,
    +                    AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT, 
SSL_CONTEXT_SERVICE, ENDPOINT_OVERRIDE,
    +                    DELIMITER, PREFIX));
    +
    +    public static final Set<Relationship> relationships = 
Collections.unmodifiableSet(
    +            new HashSet<>(Collections.singletonList(REL_SUCCESS)));
    +
    +    public static final String CURRENT_TIMESTAMP = "currentTimestamp";
    +    public static final String CURRENT_KEY_PREFIX = "key-";
    +
    +    // State tracking
    +    private long currentTimestamp = 0L;
    +    private Set<String> currentKeys;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return properties;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    private Set<String> extractKeys(final StateMap stateMap) {
    +        Set<String> keys = new HashSet<>();
    +        for (Map.Entry<String, String>  entry : 
stateMap.toMap().entrySet()) {
    +            if (entry.getKey().startsWith(CURRENT_KEY_PREFIX)) {
    +                keys.add(entry.getValue());
    +            }
    +        }
    +        return keys;
    +    }
    +
    +    private void restoreState(final ProcessContext context) throws 
IOException {
    +        final StateMap stateMap = 
context.getStateManager().getState(Scope.CLUSTER);
    +        if (stateMap.getVersion() == -1L) {
    +            currentTimestamp = 0L;
    +            currentKeys = new HashSet<>();
    +        } else {
    +            currentTimestamp = 
Long.parseLong(stateMap.get(CURRENT_TIMESTAMP));
    --- End diff --
    
    The ListS3 processor throws exceptions on restart if state was cleared 
through the UI while the processor was stopped.
    
    2016-02-22 14:00:05,586 ERROR [Timer-Driven Process Thread-3] 
org.apache.nifi.processors.aws.s3.ListS3 
ListS3[id=5763acff-1b63-461a-bb72-fd12bd7ac603] 
ListS3[id=5763acff-1b63-461a-bb72-fd12bd7ac603] failed to process due to 
java.lang.NumberFormatException: null; rolling back session: 
java.lang.NumberFormatException: null
    2016-02-22 14:00:05,637 ERROR [Timer-Driven Process Thread-3] 
org.apache.nifi.processors.aws.s3.ListS3 
    java.lang.NumberFormatException: null
        at java.lang.Long.parseLong(Long.java:404) ~[na:1.7.0_80]
        at java.lang.Long.parseLong(Long.java:483) ~[na:1.7.0_80]
        at 
org.apache.nifi.processors.aws.s3.ListS3.restoreState(ListS3.java:133) ~[na:na]
        at org.apache.nifi.processors.aws.s3.ListS3.onTrigger(ListS3.java:155) 
~[na:na]
        at 
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
 ~[nifi-api-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
        at 
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1139)
 [nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
        at 
org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:139)
 [nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
        at 
org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:49)
 [nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
        at 
org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:124)
 [nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
[na:1.7.0_80]
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304) 
[na:1.7.0_80]
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
 [na:1.7.0_80]
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
 [na:1.7.0_80]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_80]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_80]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_80]



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to