MikeThomsen commented on code in PR #5947: URL: https://github.com/apache/nifi/pull/5947#discussion_r846633486
########## nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/ConsumeTwitter.java: ########## @@ -0,0 +1,316 @@ +/* + * 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.twitter; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +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.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Set; +import java.util.List; +import java.util.HashSet; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +@SupportsBatching +@InputRequirement(Requirement.INPUT_FORBIDDEN) +@Tags({"twitter", "tweets", "social media", "status", "json"}) +@CapabilityDescription("Streams tweets from Twitter's streaming API v2. The stream provides a sample stream or a search " + + "stream based on previously uploaded rules. This processor also provides a pass through for certain fields of the " + + "tweet to be returned as part of the response. See " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/introduction for more information regarding the " + + "Tweet object model. \n\n" + + "Warning: the underlying Java SDK used is still in beta as of the publishing of this processor feature.") +@WritesAttribute(attribute = "mime.type", description = "Sets mime type to application/json") +public class ConsumeTwitter extends AbstractProcessor { + + static final AllowableValue ENDPOINT_SAMPLE = new AllowableValue("Sample Endpoint", + "Sample Endpoint", + "The endpoint that provides a stream of about 1% of tweets in real-time"); + static final AllowableValue ENDPOINT_SEARCH = new AllowableValue("Search Endpoint", + "Search Endpoint", + "The endpoint that provides a stream of tweets that matches the rules you added to the stream. " + + "If rules are not configured, then the stream will be empty"); + + public static final PropertyDescriptor ENDPOINT = new PropertyDescriptor.Builder() + .name("Twitter Endpoint") Review Comment: Name for each property should be something like `consume-twitter-endpoint`. These are not meant to be human-friendly fields. That's what `displayName` is for. ########## nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/ConsumeTwitter.java: ########## @@ -0,0 +1,316 @@ +/* + * 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.twitter; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +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.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Set; +import java.util.List; +import java.util.HashSet; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +@SupportsBatching +@InputRequirement(Requirement.INPUT_FORBIDDEN) +@Tags({"twitter", "tweets", "social media", "status", "json"}) +@CapabilityDescription("Streams tweets from Twitter's streaming API v2. The stream provides a sample stream or a search " + + "stream based on previously uploaded rules. This processor also provides a pass through for certain fields of the " + + "tweet to be returned as part of the response. See " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/introduction for more information regarding the " + + "Tweet object model. \n\n" + + "Warning: the underlying Java SDK used is still in beta as of the publishing of this processor feature.") +@WritesAttribute(attribute = "mime.type", description = "Sets mime type to application/json") +public class ConsumeTwitter extends AbstractProcessor { + + static final AllowableValue ENDPOINT_SAMPLE = new AllowableValue("Sample Endpoint", + "Sample Endpoint", + "The endpoint that provides a stream of about 1% of tweets in real-time"); + static final AllowableValue ENDPOINT_SEARCH = new AllowableValue("Search Endpoint", + "Search Endpoint", + "The endpoint that provides a stream of tweets that matches the rules you added to the stream. " + + "If rules are not configured, then the stream will be empty"); + + public static final PropertyDescriptor ENDPOINT = new PropertyDescriptor.Builder() + .name("Twitter Endpoint") + .displayName("Twitter Endpoint") + .description("Specifies which endpoint tweets should be pulled from. " + + "Usage of search endpoint requires that rules be uploaded beforehand. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/api-reference/" + + "post-tweets-search-stream-rules") + .required(true) + .allowableValues(ENDPOINT_SAMPLE, ENDPOINT_SEARCH) + .defaultValue(ENDPOINT_SAMPLE.getValue()) + .build(); + public static final PropertyDescriptor BASE_PATH = new PropertyDescriptor.Builder() + .name("Base Path") + .displayName("Base Path") + .description("Specifies which base path the API client will use for HTTP requests. " + + "Generally should not be changed from the default https://api.twitter.com except for testing") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("https://api.twitter.com") + .build(); + public static final PropertyDescriptor BEARER_TOKEN = new PropertyDescriptor.Builder() + .name("Bearer Token") + .displayName("Bearer Token") + .description("The Bearer Token provided by Twitter.") + .required(true) + .sensitive(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor QUEUE_SIZE = new PropertyDescriptor.Builder() + .name("Queue Size") + .displayName("Queue Size") + .description("The size of the queue used to queue tweets") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10000") + .build(); + public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder() + .name("Batch Size") + .displayName("Batch Size") + .description("The maximum size of the number of tweets to be written to a single FlowFile." + + "Will write less tweets if it there are not any tweets left in queue") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10") + .build(); + public static final PropertyDescriptor TWEET_FIELDS = new PropertyDescriptor.Builder() + .name("Tweet Fields") + .displayName("Tweet Fields") + .description("A comma-separated list of tweet fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/tweet " + + "for proper usage. Possible field values are: " + + "attachments, author_id, context_annotations, conversation_id, created_at, entities, geo, id, " + + "in_reply_to_user_id, lang, non_public_metrics, organic_metrics, possibly_sensitive, promoted_metrics, " + + "public_metrics, referenced_tweets, reply_settings, source, text, withheld") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor USER_FIELDS = new PropertyDescriptor.Builder() + .name("User Fields") + .displayName("User Fields") + .description("A comma-separated list of user fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/user " + + "for proper usage. Possible field values are: " + + "created_at, description, entities, id, location, name, pinned_tweet_id, profile_image_url, " + + "protected, public_metrics, url, username, verified, withheld") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor MEDIA_FIELDS = new PropertyDescriptor.Builder() + .name("Media Fields") + .displayName("Media Fields") + .description("A comma-separated list of media fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/media " + + "for proper usage. Possible field values are: " + + "alt_text, duration_ms, height, media_key, non_public_metrics, organic_metrics, preview_image_url, " + + "promoted_metrics, public_metrics, type, url, width") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor POLL_FIELDS = new PropertyDescriptor.Builder() + .name("Poll Fields") + .displayName("Poll Fields") + .description("A comma-separated list of poll fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/poll " + + "for proper usage. Possible field values are: " + + "duration_minutes, end_datetime, id, options, voting_status") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor PLACE_FIELDS = new PropertyDescriptor.Builder() + .name("Place Fields") + .displayName("Place Fields") + .description("A comma-separated list of place fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/place " + + "for proper usage. Possible field values are: " + + "contained_within, country, country_code, full_name, geo, id, name, place_type") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor EXPANSIONS = new PropertyDescriptor.Builder() + .name("Expansions") + .displayName("Expansions") + .description("A comma-separated list of expansions for objects in the returned tweet. See " + + "https://developer.twitter.com/en/docs/twitter-api/expansions " + + "for proper usage. Possible field values are: " + + "author_id, referenced_tweets.id, referenced_tweets.id.author_id, entities.mentions.username, " + + "attachments.poll_ids, attachments.media_keys ,in_reply_to_user_id, geo.place_id") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor BACKFILL_MINUTES = new PropertyDescriptor.Builder() + .name("Backfill Minutes") + .displayName("Backfill Minutes") + .description("The number of minutes (up to 5 minutes) of streaming data to be requested after a " + + "disconnect. Only available for project with academic research access. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/integrate/" + + "recovery-and-redundancy-features") + .required(true) + .defaultValue("0") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("All tweets will be routed to this relationship.") + .build(); + + private List<PropertyDescriptor> descriptors; + private Set<Relationship> relationships; + + private TwitterStreamAPI api; + + private volatile BlockingQueue<String> messageQueue; + + private void emptyQueue() { + while (!messageQueue.isEmpty()) { + messageQueue.poll(); + } + } + + @Override + protected void init(ProcessorInitializationContext context) { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(ENDPOINT); + descriptors.add(BASE_PATH); + descriptors.add(BEARER_TOKEN); + descriptors.add(QUEUE_SIZE); + descriptors.add(BATCH_SIZE); + descriptors.add(TWEET_FIELDS); + descriptors.add(USER_FIELDS); + descriptors.add(MEDIA_FIELDS); + descriptors.add(POLL_FIELDS); + descriptors.add(PLACE_FIELDS); + descriptors.add(EXPANSIONS); + descriptors.add(BACKFILL_MINUTES); + + this.descriptors = Collections.unmodifiableList(descriptors); + + final Set<Relationship> relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + public Set<Relationship> getRelationships() { + return this.relationships; + } + + @Override + public final List<PropertyDescriptor> getSupportedPropertyDescriptors() { + return this.descriptors; + } + + + @OnScheduled + public void onScheduled(final ProcessContext context) { + messageQueue = new LinkedBlockingQueue<>(context.getProperty(QUEUE_SIZE).asInteger()); + + api = new TwitterStreamAPI(context, messageQueue, getLogger()); + api.setBasePath(context.getProperty(BASE_PATH).getValue()); + final String endpointName = context.getProperty(ENDPOINT).getValue(); + if (ENDPOINT_SAMPLE.getValue().equals(endpointName)) { + api.start(TwitterStreamAPI.SAMPLE_ENDPOINT); + } else if (ENDPOINT_SEARCH.getValue().equals(endpointName)) { + api.start(TwitterStreamAPI.SEARCH_ENDPOINT); + } else { + throw new AssertionError("Endpoint was invalid value: " + endpointName); + } + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + if (messageQueue.isEmpty()) { + context.yield(); + return; + } + + FlowFile flowFile = session.create(); + flowFile = session.write(flowFile, new OutputStreamCallback() { Review Comment: It would be safer to use Jackson to convert the incoming tweets into `Map<String, Object>` types and then turn the current queue contents into a stringified array with Jackson. ########## nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/ConsumeTwitter.java: ########## @@ -0,0 +1,316 @@ +/* + * 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.twitter; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +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.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Set; +import java.util.List; +import java.util.HashSet; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +@SupportsBatching +@InputRequirement(Requirement.INPUT_FORBIDDEN) +@Tags({"twitter", "tweets", "social media", "status", "json"}) +@CapabilityDescription("Streams tweets from Twitter's streaming API v2. The stream provides a sample stream or a search " + + "stream based on previously uploaded rules. This processor also provides a pass through for certain fields of the " + + "tweet to be returned as part of the response. See " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/introduction for more information regarding the " + + "Tweet object model. \n\n" + + "Warning: the underlying Java SDK used is still in beta as of the publishing of this processor feature.") +@WritesAttribute(attribute = "mime.type", description = "Sets mime type to application/json") +public class ConsumeTwitter extends AbstractProcessor { + + static final AllowableValue ENDPOINT_SAMPLE = new AllowableValue("Sample Endpoint", + "Sample Endpoint", + "The endpoint that provides a stream of about 1% of tweets in real-time"); + static final AllowableValue ENDPOINT_SEARCH = new AllowableValue("Search Endpoint", + "Search Endpoint", + "The endpoint that provides a stream of tweets that matches the rules you added to the stream. " + + "If rules are not configured, then the stream will be empty"); + + public static final PropertyDescriptor ENDPOINT = new PropertyDescriptor.Builder() + .name("Twitter Endpoint") + .displayName("Twitter Endpoint") + .description("Specifies which endpoint tweets should be pulled from. " + + "Usage of search endpoint requires that rules be uploaded beforehand. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/api-reference/" + + "post-tweets-search-stream-rules") + .required(true) + .allowableValues(ENDPOINT_SAMPLE, ENDPOINT_SEARCH) + .defaultValue(ENDPOINT_SAMPLE.getValue()) + .build(); + public static final PropertyDescriptor BASE_PATH = new PropertyDescriptor.Builder() + .name("Base Path") + .displayName("Base Path") + .description("Specifies which base path the API client will use for HTTP requests. " + + "Generally should not be changed from the default https://api.twitter.com except for testing") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("https://api.twitter.com") + .build(); + public static final PropertyDescriptor BEARER_TOKEN = new PropertyDescriptor.Builder() + .name("Bearer Token") + .displayName("Bearer Token") + .description("The Bearer Token provided by Twitter.") + .required(true) + .sensitive(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor QUEUE_SIZE = new PropertyDescriptor.Builder() + .name("Queue Size") + .displayName("Queue Size") + .description("The size of the queue used to queue tweets") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10000") + .build(); + public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder() + .name("Batch Size") + .displayName("Batch Size") + .description("The maximum size of the number of tweets to be written to a single FlowFile." + + "Will write less tweets if it there are not any tweets left in queue") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10") + .build(); + public static final PropertyDescriptor TWEET_FIELDS = new PropertyDescriptor.Builder() + .name("Tweet Fields") + .displayName("Tweet Fields") + .description("A comma-separated list of tweet fields to be returned as part of the tweet. Refer to " + Review Comment: Should have a default value ########## nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/ConsumeTwitter.java: ########## @@ -0,0 +1,316 @@ +/* + * 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.twitter; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +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.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Set; +import java.util.List; +import java.util.HashSet; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +@SupportsBatching +@InputRequirement(Requirement.INPUT_FORBIDDEN) +@Tags({"twitter", "tweets", "social media", "status", "json"}) +@CapabilityDescription("Streams tweets from Twitter's streaming API v2. The stream provides a sample stream or a search " + + "stream based on previously uploaded rules. This processor also provides a pass through for certain fields of the " + + "tweet to be returned as part of the response. See " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/introduction for more information regarding the " + + "Tweet object model. \n\n" + + "Warning: the underlying Java SDK used is still in beta as of the publishing of this processor feature.") +@WritesAttribute(attribute = "mime.type", description = "Sets mime type to application/json") +public class ConsumeTwitter extends AbstractProcessor { + + static final AllowableValue ENDPOINT_SAMPLE = new AllowableValue("Sample Endpoint", + "Sample Endpoint", + "The endpoint that provides a stream of about 1% of tweets in real-time"); + static final AllowableValue ENDPOINT_SEARCH = new AllowableValue("Search Endpoint", + "Search Endpoint", + "The endpoint that provides a stream of tweets that matches the rules you added to the stream. " + + "If rules are not configured, then the stream will be empty"); + + public static final PropertyDescriptor ENDPOINT = new PropertyDescriptor.Builder() + .name("Twitter Endpoint") + .displayName("Twitter Endpoint") + .description("Specifies which endpoint tweets should be pulled from. " + + "Usage of search endpoint requires that rules be uploaded beforehand. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/api-reference/" + + "post-tweets-search-stream-rules") + .required(true) + .allowableValues(ENDPOINT_SAMPLE, ENDPOINT_SEARCH) + .defaultValue(ENDPOINT_SAMPLE.getValue()) + .build(); + public static final PropertyDescriptor BASE_PATH = new PropertyDescriptor.Builder() + .name("Base Path") + .displayName("Base Path") + .description("Specifies which base path the API client will use for HTTP requests. " + + "Generally should not be changed from the default https://api.twitter.com except for testing") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("https://api.twitter.com") + .build(); + public static final PropertyDescriptor BEARER_TOKEN = new PropertyDescriptor.Builder() + .name("Bearer Token") + .displayName("Bearer Token") + .description("The Bearer Token provided by Twitter.") + .required(true) + .sensitive(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor QUEUE_SIZE = new PropertyDescriptor.Builder() + .name("Queue Size") + .displayName("Queue Size") + .description("The size of the queue used to queue tweets") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10000") + .build(); + public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder() + .name("Batch Size") + .displayName("Batch Size") + .description("The maximum size of the number of tweets to be written to a single FlowFile." + + "Will write less tweets if it there are not any tweets left in queue") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10") + .build(); + public static final PropertyDescriptor TWEET_FIELDS = new PropertyDescriptor.Builder() + .name("Tweet Fields") + .displayName("Tweet Fields") + .description("A comma-separated list of tweet fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/tweet " + + "for proper usage. Possible field values are: " + + "attachments, author_id, context_annotations, conversation_id, created_at, entities, geo, id, " + + "in_reply_to_user_id, lang, non_public_metrics, organic_metrics, possibly_sensitive, promoted_metrics, " + + "public_metrics, referenced_tweets, reply_settings, source, text, withheld") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor USER_FIELDS = new PropertyDescriptor.Builder() + .name("User Fields") + .displayName("User Fields") + .description("A comma-separated list of user fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/user " + + "for proper usage. Possible field values are: " + + "created_at, description, entities, id, location, name, pinned_tweet_id, profile_image_url, " + + "protected, public_metrics, url, username, verified, withheld") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor MEDIA_FIELDS = new PropertyDescriptor.Builder() + .name("Media Fields") Review Comment: All of the *_FIELDS should have defaults. ########## nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/TwitterStreamAPI.java: ########## @@ -0,0 +1,173 @@ +/* + * 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.twitter; + +import com.twitter.clientlib.ApiException; +import com.twitter.clientlib.TwitterCredentialsBearer; +import com.twitter.clientlib.api.TwitterApi; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +public class TwitterStreamAPI { + public static final String SEARCH_ENDPOINT = "Search Endpoint"; + public static final String SAMPLE_ENDPOINT = "Sample Endpoint"; + public static final String SAMPLE_PATH = "/2/tweets/sample/stream"; + public static final String SEARCH_PATH = "/2/tweets/search/stream"; + + private static final String BEARER_TOKEN_PROPERTY_NAME = "Bearer Token"; + private static final String TWEET_FIELDS_PROPERTY_NAME = "Tweet Fields"; + private static final String USER_FIELDS_PROPERTY_NAME = "User Fields"; + private static final String MEDIA_FIELDS_PROPERTY_NAME = "Media Fields"; + private static final String POLL_FIELDS_PROPERTY_NAME = "Poll Fields"; + private static final String PLACE_FIELDS_PROPERTY_NAME = "Place Fields"; + private static final String EXPANSIONS_PROPERTY_NAME = "Expansions"; + private static final String BACKFILL_MINUTES_PROPERTY_NAME = "Backfill Minutes"; + + private final BlockingQueue<String> queue; + private final ComponentLog logger; + + private final ExecutorService executorService; + + private final Set<String> tweetFields; + private final Set<String> userFields; + private final Set<String> mediaFields; + private final Set<String> pollFields; + private final Set<String> placeFields; + private final Set<String> expansions; + private final int backfillMinutes; + private final TwitterApi api; + private InputStream stream; + + private Set<String> parseCommaSeparatedListPropreties(final ProcessContext context, final String propertyName) { + Set<String> fields = null; + if (context.getProperty(propertyName).isSet()) { + fields = new HashSet<>(); + final String fieldsString = context.getProperty(propertyName).getValue(); + for (final String field: fieldsString.split(",")) { + fields.add(field.trim()); + } + } + return fields; + } + + public TwitterStreamAPI(final ProcessContext context, final BlockingQueue<String> queue, final ComponentLog logger) { + assert context != null; + assert queue != null; + assert logger != null; + + this.queue = queue; + this.logger = logger; + + this.tweetFields = parseCommaSeparatedListPropreties(context, TWEET_FIELDS_PROPERTY_NAME); + this.userFields = parseCommaSeparatedListPropreties(context, USER_FIELDS_PROPERTY_NAME); + this.mediaFields = parseCommaSeparatedListPropreties(context, MEDIA_FIELDS_PROPERTY_NAME); + this.pollFields = parseCommaSeparatedListPropreties(context, POLL_FIELDS_PROPERTY_NAME); + this.placeFields = parseCommaSeparatedListPropreties(context, PLACE_FIELDS_PROPERTY_NAME); + this.expansions = parseCommaSeparatedListPropreties(context, EXPANSIONS_PROPERTY_NAME); + this.backfillMinutes = context.getProperty(BACKFILL_MINUTES_PROPERTY_NAME).asInteger(); + + TwitterCredentialsBearer creds = new TwitterCredentialsBearer(context.getProperty(BEARER_TOKEN_PROPERTY_NAME).getValue()); + api = new TwitterApi(); + api.setTwitterCredentials(creds); + + this.executorService = Executors.newSingleThreadExecutor(); + } + + public String getBasePath() { + return api.getApiClient().getBasePath(); + } + + public void setBasePath(final String path) { + api.getApiClient().setBasePath(path); + } + + /** + * This method would be called when we would like the stream to get started. This method will spin off a thread that + * will continue to queue tweets on to the given queue passed in the constructor. The thread will continue + * to run until {@code stop} is called. + * @param endpoint {@code TwitterStreamAPI.SAMPLE_ENDPOINT} or {@code TwitterStreamAPI.SEARCH_ENDPOINT} + */ + public void start(final String endpoint) { + try { + if (endpoint.equals(SAMPLE_ENDPOINT)) { + stream = api.tweets().sampleStream(expansions, tweetFields, userFields, mediaFields, placeFields, pollFields, backfillMinutes); + } else if (endpoint.equals(SEARCH_ENDPOINT)) { + stream = api.tweets().searchStream(expansions, tweetFields, userFields, mediaFields, placeFields, pollFields, backfillMinutes); + } else { + throw new AssertionError("Endpoint was invalid value: " + endpoint); Review Comment: Use `ProcessException` ########## nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/ConsumeTwitter.java: ########## @@ -0,0 +1,316 @@ +/* + * 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.twitter; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +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.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Set; +import java.util.List; +import java.util.HashSet; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +@SupportsBatching +@InputRequirement(Requirement.INPUT_FORBIDDEN) +@Tags({"twitter", "tweets", "social media", "status", "json"}) +@CapabilityDescription("Streams tweets from Twitter's streaming API v2. The stream provides a sample stream or a search " + + "stream based on previously uploaded rules. This processor also provides a pass through for certain fields of the " + + "tweet to be returned as part of the response. See " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/introduction for more information regarding the " + + "Tweet object model. \n\n" + + "Warning: the underlying Java SDK used is still in beta as of the publishing of this processor feature.") +@WritesAttribute(attribute = "mime.type", description = "Sets mime type to application/json") +public class ConsumeTwitter extends AbstractProcessor { + + static final AllowableValue ENDPOINT_SAMPLE = new AllowableValue("Sample Endpoint", + "Sample Endpoint", + "The endpoint that provides a stream of about 1% of tweets in real-time"); + static final AllowableValue ENDPOINT_SEARCH = new AllowableValue("Search Endpoint", + "Search Endpoint", + "The endpoint that provides a stream of tweets that matches the rules you added to the stream. " + + "If rules are not configured, then the stream will be empty"); + + public static final PropertyDescriptor ENDPOINT = new PropertyDescriptor.Builder() + .name("Twitter Endpoint") + .displayName("Twitter Endpoint") + .description("Specifies which endpoint tweets should be pulled from. " + + "Usage of search endpoint requires that rules be uploaded beforehand. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/api-reference/" + + "post-tweets-search-stream-rules") + .required(true) + .allowableValues(ENDPOINT_SAMPLE, ENDPOINT_SEARCH) + .defaultValue(ENDPOINT_SAMPLE.getValue()) + .build(); + public static final PropertyDescriptor BASE_PATH = new PropertyDescriptor.Builder() + .name("Base Path") + .displayName("Base Path") + .description("Specifies which base path the API client will use for HTTP requests. " + + "Generally should not be changed from the default https://api.twitter.com except for testing") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("https://api.twitter.com") + .build(); + public static final PropertyDescriptor BEARER_TOKEN = new PropertyDescriptor.Builder() Review Comment: What's the expiration policy for Twitter bearer tokens and does Twitter provide easily accessed OAuth2 APIs to get a token? ########## nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/ConsumeTwitter.java: ########## @@ -0,0 +1,316 @@ +/* + * 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.twitter; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +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.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Set; +import java.util.List; +import java.util.HashSet; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +@SupportsBatching +@InputRequirement(Requirement.INPUT_FORBIDDEN) +@Tags({"twitter", "tweets", "social media", "status", "json"}) +@CapabilityDescription("Streams tweets from Twitter's streaming API v2. The stream provides a sample stream or a search " + + "stream based on previously uploaded rules. This processor also provides a pass through for certain fields of the " + + "tweet to be returned as part of the response. See " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/introduction for more information regarding the " + + "Tweet object model. \n\n" + + "Warning: the underlying Java SDK used is still in beta as of the publishing of this processor feature.") +@WritesAttribute(attribute = "mime.type", description = "Sets mime type to application/json") +public class ConsumeTwitter extends AbstractProcessor { + + static final AllowableValue ENDPOINT_SAMPLE = new AllowableValue("Sample Endpoint", + "Sample Endpoint", + "The endpoint that provides a stream of about 1% of tweets in real-time"); + static final AllowableValue ENDPOINT_SEARCH = new AllowableValue("Search Endpoint", + "Search Endpoint", + "The endpoint that provides a stream of tweets that matches the rules you added to the stream. " + + "If rules are not configured, then the stream will be empty"); + + public static final PropertyDescriptor ENDPOINT = new PropertyDescriptor.Builder() + .name("Twitter Endpoint") + .displayName("Twitter Endpoint") + .description("Specifies which endpoint tweets should be pulled from. " + + "Usage of search endpoint requires that rules be uploaded beforehand. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/api-reference/" + + "post-tweets-search-stream-rules") + .required(true) + .allowableValues(ENDPOINT_SAMPLE, ENDPOINT_SEARCH) + .defaultValue(ENDPOINT_SAMPLE.getValue()) + .build(); + public static final PropertyDescriptor BASE_PATH = new PropertyDescriptor.Builder() + .name("Base Path") + .displayName("Base Path") + .description("Specifies which base path the API client will use for HTTP requests. " + + "Generally should not be changed from the default https://api.twitter.com except for testing") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("https://api.twitter.com") + .build(); + public static final PropertyDescriptor BEARER_TOKEN = new PropertyDescriptor.Builder() + .name("Bearer Token") + .displayName("Bearer Token") + .description("The Bearer Token provided by Twitter.") + .required(true) + .sensitive(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor QUEUE_SIZE = new PropertyDescriptor.Builder() + .name("Queue Size") + .displayName("Queue Size") + .description("The size of the queue used to queue tweets") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10000") + .build(); + public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder() + .name("Batch Size") + .displayName("Batch Size") + .description("The maximum size of the number of tweets to be written to a single FlowFile." + + "Will write less tweets if it there are not any tweets left in queue") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10") + .build(); + public static final PropertyDescriptor TWEET_FIELDS = new PropertyDescriptor.Builder() + .name("Tweet Fields") + .displayName("Tweet Fields") + .description("A comma-separated list of tweet fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/tweet " + + "for proper usage. Possible field values are: " + + "attachments, author_id, context_annotations, conversation_id, created_at, entities, geo, id, " + + "in_reply_to_user_id, lang, non_public_metrics, organic_metrics, possibly_sensitive, promoted_metrics, " + + "public_metrics, referenced_tweets, reply_settings, source, text, withheld") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor USER_FIELDS = new PropertyDescriptor.Builder() + .name("User Fields") + .displayName("User Fields") + .description("A comma-separated list of user fields to be returned as part of the tweet. Refer to " + Review Comment: Should have a default value ########## nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/ConsumeTwitter.java: ########## @@ -0,0 +1,316 @@ +/* + * 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.twitter; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +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.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Set; +import java.util.List; +import java.util.HashSet; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +@SupportsBatching +@InputRequirement(Requirement.INPUT_FORBIDDEN) +@Tags({"twitter", "tweets", "social media", "status", "json"}) +@CapabilityDescription("Streams tweets from Twitter's streaming API v2. The stream provides a sample stream or a search " + + "stream based on previously uploaded rules. This processor also provides a pass through for certain fields of the " + + "tweet to be returned as part of the response. See " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/introduction for more information regarding the " + + "Tweet object model. \n\n" + + "Warning: the underlying Java SDK used is still in beta as of the publishing of this processor feature.") +@WritesAttribute(attribute = "mime.type", description = "Sets mime type to application/json") +public class ConsumeTwitter extends AbstractProcessor { + + static final AllowableValue ENDPOINT_SAMPLE = new AllowableValue("Sample Endpoint", + "Sample Endpoint", + "The endpoint that provides a stream of about 1% of tweets in real-time"); + static final AllowableValue ENDPOINT_SEARCH = new AllowableValue("Search Endpoint", + "Search Endpoint", + "The endpoint that provides a stream of tweets that matches the rules you added to the stream. " + + "If rules are not configured, then the stream will be empty"); + + public static final PropertyDescriptor ENDPOINT = new PropertyDescriptor.Builder() + .name("Twitter Endpoint") + .displayName("Twitter Endpoint") + .description("Specifies which endpoint tweets should be pulled from. " + + "Usage of search endpoint requires that rules be uploaded beforehand. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/api-reference/" + + "post-tweets-search-stream-rules") + .required(true) + .allowableValues(ENDPOINT_SAMPLE, ENDPOINT_SEARCH) + .defaultValue(ENDPOINT_SAMPLE.getValue()) + .build(); + public static final PropertyDescriptor BASE_PATH = new PropertyDescriptor.Builder() + .name("Base Path") + .displayName("Base Path") + .description("Specifies which base path the API client will use for HTTP requests. " + + "Generally should not be changed from the default https://api.twitter.com except for testing") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("https://api.twitter.com") + .build(); + public static final PropertyDescriptor BEARER_TOKEN = new PropertyDescriptor.Builder() + .name("Bearer Token") + .displayName("Bearer Token") + .description("The Bearer Token provided by Twitter.") + .required(true) + .sensitive(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor QUEUE_SIZE = new PropertyDescriptor.Builder() + .name("Queue Size") + .displayName("Queue Size") + .description("The size of the queue used to queue tweets") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10000") + .build(); + public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder() + .name("Batch Size") + .displayName("Batch Size") + .description("The maximum size of the number of tweets to be written to a single FlowFile." + + "Will write less tweets if it there are not any tweets left in queue") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10") + .build(); + public static final PropertyDescriptor TWEET_FIELDS = new PropertyDescriptor.Builder() + .name("Tweet Fields") + .displayName("Tweet Fields") + .description("A comma-separated list of tweet fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/tweet " + + "for proper usage. Possible field values are: " + + "attachments, author_id, context_annotations, conversation_id, created_at, entities, geo, id, " + + "in_reply_to_user_id, lang, non_public_metrics, organic_metrics, possibly_sensitive, promoted_metrics, " + + "public_metrics, referenced_tweets, reply_settings, source, text, withheld") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor USER_FIELDS = new PropertyDescriptor.Builder() + .name("User Fields") + .displayName("User Fields") + .description("A comma-separated list of user fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/user " + + "for proper usage. Possible field values are: " + + "created_at, description, entities, id, location, name, pinned_tweet_id, profile_image_url, " + + "protected, public_metrics, url, username, verified, withheld") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor MEDIA_FIELDS = new PropertyDescriptor.Builder() + .name("Media Fields") + .displayName("Media Fields") + .description("A comma-separated list of media fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/media " + + "for proper usage. Possible field values are: " + + "alt_text, duration_ms, height, media_key, non_public_metrics, organic_metrics, preview_image_url, " + + "promoted_metrics, public_metrics, type, url, width") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor POLL_FIELDS = new PropertyDescriptor.Builder() + .name("Poll Fields") + .displayName("Poll Fields") + .description("A comma-separated list of poll fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/poll " + + "for proper usage. Possible field values are: " + + "duration_minutes, end_datetime, id, options, voting_status") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor PLACE_FIELDS = new PropertyDescriptor.Builder() + .name("Place Fields") + .displayName("Place Fields") + .description("A comma-separated list of place fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/place " + + "for proper usage. Possible field values are: " + + "contained_within, country, country_code, full_name, geo, id, name, place_type") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor EXPANSIONS = new PropertyDescriptor.Builder() + .name("Expansions") + .displayName("Expansions") + .description("A comma-separated list of expansions for objects in the returned tweet. See " + + "https://developer.twitter.com/en/docs/twitter-api/expansions " + + "for proper usage. Possible field values are: " + + "author_id, referenced_tweets.id, referenced_tweets.id.author_id, entities.mentions.username, " + + "attachments.poll_ids, attachments.media_keys ,in_reply_to_user_id, geo.place_id") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor BACKFILL_MINUTES = new PropertyDescriptor.Builder() + .name("Backfill Minutes") + .displayName("Backfill Minutes") + .description("The number of minutes (up to 5 minutes) of streaming data to be requested after a " + + "disconnect. Only available for project with academic research access. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/integrate/" + + "recovery-and-redundancy-features") + .required(true) + .defaultValue("0") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("All tweets will be routed to this relationship.") + .build(); + + private List<PropertyDescriptor> descriptors; + private Set<Relationship> relationships; + + private TwitterStreamAPI api; + + private volatile BlockingQueue<String> messageQueue; + + private void emptyQueue() { + while (!messageQueue.isEmpty()) { + messageQueue.poll(); + } + } + + @Override + protected void init(ProcessorInitializationContext context) { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(ENDPOINT); + descriptors.add(BASE_PATH); + descriptors.add(BEARER_TOKEN); + descriptors.add(QUEUE_SIZE); + descriptors.add(BATCH_SIZE); + descriptors.add(TWEET_FIELDS); + descriptors.add(USER_FIELDS); + descriptors.add(MEDIA_FIELDS); + descriptors.add(POLL_FIELDS); + descriptors.add(PLACE_FIELDS); + descriptors.add(EXPANSIONS); + descriptors.add(BACKFILL_MINUTES); + + this.descriptors = Collections.unmodifiableList(descriptors); + + final Set<Relationship> relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + public Set<Relationship> getRelationships() { + return this.relationships; + } + + @Override + public final List<PropertyDescriptor> getSupportedPropertyDescriptors() { + return this.descriptors; + } + + + @OnScheduled + public void onScheduled(final ProcessContext context) { + messageQueue = new LinkedBlockingQueue<>(context.getProperty(QUEUE_SIZE).asInteger()); + + api = new TwitterStreamAPI(context, messageQueue, getLogger()); + api.setBasePath(context.getProperty(BASE_PATH).getValue()); + final String endpointName = context.getProperty(ENDPOINT).getValue(); + if (ENDPOINT_SAMPLE.getValue().equals(endpointName)) { + api.start(TwitterStreamAPI.SAMPLE_ENDPOINT); + } else if (ENDPOINT_SEARCH.getValue().equals(endpointName)) { + api.start(TwitterStreamAPI.SEARCH_ENDPOINT); + } else { + throw new AssertionError("Endpoint was invalid value: " + endpointName); + } + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + if (messageQueue.isEmpty()) { + context.yield(); + return; + } + + FlowFile flowFile = session.create(); + flowFile = session.write(flowFile, new OutputStreamCallback() { + @Override + public void process(OutputStream out) throws IOException { + out.write("[".getBytes(StandardCharsets.UTF_8)); + final int batchSize = context.getProperty(BATCH_SIZE).asInteger(); + String tweet = messageQueue.poll(); + out.write(tweet.getBytes(StandardCharsets.UTF_8)); + int tweetCount = 1; + while (tweetCount < batchSize && !messageQueue.isEmpty()) { + out.write(",".getBytes(StandardCharsets.UTF_8)); + tweet = messageQueue.poll(); + out.write(tweet.getBytes(StandardCharsets.UTF_8)); + tweetCount++; + } + out.write("]".getBytes(StandardCharsets.UTF_8)); + } + }); + + final Map<String, String> attributes = new HashMap<>(); + attributes.put(CoreAttributes.MIME_TYPE.key(), "application/json"); + attributes.put(CoreAttributes.FILENAME.key(), flowFile.getAttribute(CoreAttributes.FILENAME.key()) + ".json"); + flowFile = session.putAllAttributes(flowFile, attributes); + + session.transfer(flowFile, REL_SUCCESS); + + String transitUri = api.getBasePath(); + final String endpointName = context.getProperty(ENDPOINT).getValue(); + if (ENDPOINT_SAMPLE.getValue().equals(endpointName)) { + transitUri += TwitterStreamAPI.SAMPLE_PATH; + } else if (ENDPOINT_SEARCH.getValue().equals(endpointName)) { + transitUri += TwitterStreamAPI.SEARCH_PATH; + } else { + throw new AssertionError("Endpoint was invalid value: " + endpointName); Review Comment: Should be a `ProcessException`. ########## nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/ConsumeTwitter.java: ########## @@ -0,0 +1,316 @@ +/* + * 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.twitter; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +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.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Set; +import java.util.List; +import java.util.HashSet; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +@SupportsBatching +@InputRequirement(Requirement.INPUT_FORBIDDEN) +@Tags({"twitter", "tweets", "social media", "status", "json"}) +@CapabilityDescription("Streams tweets from Twitter's streaming API v2. The stream provides a sample stream or a search " + + "stream based on previously uploaded rules. This processor also provides a pass through for certain fields of the " + + "tweet to be returned as part of the response. See " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/introduction for more information regarding the " + + "Tweet object model. \n\n" + + "Warning: the underlying Java SDK used is still in beta as of the publishing of this processor feature.") +@WritesAttribute(attribute = "mime.type", description = "Sets mime type to application/json") +public class ConsumeTwitter extends AbstractProcessor { + + static final AllowableValue ENDPOINT_SAMPLE = new AllowableValue("Sample Endpoint", + "Sample Endpoint", + "The endpoint that provides a stream of about 1% of tweets in real-time"); + static final AllowableValue ENDPOINT_SEARCH = new AllowableValue("Search Endpoint", + "Search Endpoint", + "The endpoint that provides a stream of tweets that matches the rules you added to the stream. " + + "If rules are not configured, then the stream will be empty"); + + public static final PropertyDescriptor ENDPOINT = new PropertyDescriptor.Builder() + .name("Twitter Endpoint") + .displayName("Twitter Endpoint") + .description("Specifies which endpoint tweets should be pulled from. " + + "Usage of search endpoint requires that rules be uploaded beforehand. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/api-reference/" + + "post-tweets-search-stream-rules") + .required(true) + .allowableValues(ENDPOINT_SAMPLE, ENDPOINT_SEARCH) + .defaultValue(ENDPOINT_SAMPLE.getValue()) + .build(); + public static final PropertyDescriptor BASE_PATH = new PropertyDescriptor.Builder() + .name("Base Path") + .displayName("Base Path") + .description("Specifies which base path the API client will use for HTTP requests. " + + "Generally should not be changed from the default https://api.twitter.com except for testing") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("https://api.twitter.com") + .build(); + public static final PropertyDescriptor BEARER_TOKEN = new PropertyDescriptor.Builder() + .name("Bearer Token") + .displayName("Bearer Token") + .description("The Bearer Token provided by Twitter.") + .required(true) + .sensitive(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor QUEUE_SIZE = new PropertyDescriptor.Builder() + .name("Queue Size") + .displayName("Queue Size") + .description("The size of the queue used to queue tweets") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10000") + .build(); + public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder() + .name("Batch Size") + .displayName("Batch Size") + .description("The maximum size of the number of tweets to be written to a single FlowFile." + + "Will write less tweets if it there are not any tweets left in queue") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10") + .build(); + public static final PropertyDescriptor TWEET_FIELDS = new PropertyDescriptor.Builder() + .name("Tweet Fields") + .displayName("Tweet Fields") + .description("A comma-separated list of tweet fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/tweet " + + "for proper usage. Possible field values are: " + + "attachments, author_id, context_annotations, conversation_id, created_at, entities, geo, id, " + + "in_reply_to_user_id, lang, non_public_metrics, organic_metrics, possibly_sensitive, promoted_metrics, " + + "public_metrics, referenced_tweets, reply_settings, source, text, withheld") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor USER_FIELDS = new PropertyDescriptor.Builder() + .name("User Fields") + .displayName("User Fields") + .description("A comma-separated list of user fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/user " + + "for proper usage. Possible field values are: " + + "created_at, description, entities, id, location, name, pinned_tweet_id, profile_image_url, " + + "protected, public_metrics, url, username, verified, withheld") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor MEDIA_FIELDS = new PropertyDescriptor.Builder() + .name("Media Fields") + .displayName("Media Fields") + .description("A comma-separated list of media fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/media " + + "for proper usage. Possible field values are: " + + "alt_text, duration_ms, height, media_key, non_public_metrics, organic_metrics, preview_image_url, " + + "promoted_metrics, public_metrics, type, url, width") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor POLL_FIELDS = new PropertyDescriptor.Builder() + .name("Poll Fields") + .displayName("Poll Fields") + .description("A comma-separated list of poll fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/poll " + + "for proper usage. Possible field values are: " + + "duration_minutes, end_datetime, id, options, voting_status") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor PLACE_FIELDS = new PropertyDescriptor.Builder() + .name("Place Fields") + .displayName("Place Fields") + .description("A comma-separated list of place fields to be returned as part of the tweet. Refer to " + + "https://developer.twitter.com/en/docs/twitter-api/data-dictionary/object-model/place " + + "for proper usage. Possible field values are: " + + "contained_within, country, country_code, full_name, geo, id, name, place_type") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor EXPANSIONS = new PropertyDescriptor.Builder() + .name("Expansions") + .displayName("Expansions") + .description("A comma-separated list of expansions for objects in the returned tweet. See " + + "https://developer.twitter.com/en/docs/twitter-api/expansions " + + "for proper usage. Possible field values are: " + + "author_id, referenced_tweets.id, referenced_tweets.id.author_id, entities.mentions.username, " + + "attachments.poll_ids, attachments.media_keys ,in_reply_to_user_id, geo.place_id") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor BACKFILL_MINUTES = new PropertyDescriptor.Builder() + .name("Backfill Minutes") + .displayName("Backfill Minutes") + .description("The number of minutes (up to 5 minutes) of streaming data to be requested after a " + + "disconnect. Only available for project with academic research access. See " + + "https://developer.twitter.com/en/docs/twitter-api/tweets/filtered-stream/integrate/" + + "recovery-and-redundancy-features") + .required(true) + .defaultValue("0") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("All tweets will be routed to this relationship.") + .build(); + + private List<PropertyDescriptor> descriptors; + private Set<Relationship> relationships; + + private TwitterStreamAPI api; + + private volatile BlockingQueue<String> messageQueue; + + private void emptyQueue() { + while (!messageQueue.isEmpty()) { + messageQueue.poll(); + } + } + + @Override + protected void init(ProcessorInitializationContext context) { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(ENDPOINT); + descriptors.add(BASE_PATH); + descriptors.add(BEARER_TOKEN); + descriptors.add(QUEUE_SIZE); + descriptors.add(BATCH_SIZE); + descriptors.add(TWEET_FIELDS); + descriptors.add(USER_FIELDS); + descriptors.add(MEDIA_FIELDS); + descriptors.add(POLL_FIELDS); + descriptors.add(PLACE_FIELDS); + descriptors.add(EXPANSIONS); + descriptors.add(BACKFILL_MINUTES); + + this.descriptors = Collections.unmodifiableList(descriptors); + + final Set<Relationship> relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + public Set<Relationship> getRelationships() { + return this.relationships; + } + + @Override + public final List<PropertyDescriptor> getSupportedPropertyDescriptors() { + return this.descriptors; + } + + + @OnScheduled + public void onScheduled(final ProcessContext context) { + messageQueue = new LinkedBlockingQueue<>(context.getProperty(QUEUE_SIZE).asInteger()); + + api = new TwitterStreamAPI(context, messageQueue, getLogger()); + api.setBasePath(context.getProperty(BASE_PATH).getValue()); + final String endpointName = context.getProperty(ENDPOINT).getValue(); + if (ENDPOINT_SAMPLE.getValue().equals(endpointName)) { + api.start(TwitterStreamAPI.SAMPLE_ENDPOINT); + } else if (ENDPOINT_SEARCH.getValue().equals(endpointName)) { + api.start(TwitterStreamAPI.SEARCH_ENDPOINT); + } else { + throw new AssertionError("Endpoint was invalid value: " + endpointName); Review Comment: Use `ProcessException` -- 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]
