-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-12266] Adds a connector that streams data from twitter #14531
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
7a3864a
# This is a combination of 2 commits.
usingh83 eba1839
# This is a combination of 2 commits.
usingh83 0056c3d
Unit test for multiple config added, and beautification
usingh83 0e5a504
Spotless apply fixed
usingh83 2d78ad9
Removing redundant comments
usingh83 fab6391
Removing newly added test
usingh83 e1bded8
adding newly added test back
usingh83 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
41 changes: 41 additions & 0 deletions
41
...rc/main/java/org/apache/beam/examples/complete/twitterstreamgenerator/README.md
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,41 @@ | ||
| <!-- | ||
| 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. | ||
| --> | ||
|
|
||
| # Twitter Connector | ||
|
|
||
| This directory contains an example pipelines for how to perform continues stream of data from twitter streaming api ( or any other 3rd party API ). This include: | ||
|
|
||
| <ul> | ||
| <li><a href="https://javadoc.io/static/org.apache.beam/beam-sdks-java-core/current/org/apache/beam/sdk/transforms/DoFn.ProcessElement.html">Splitable Dofn</a> | ||
| — A simple example of implementation of splittable dofn on an unbounded source with a simple incrementing watermarking logic.</li> | ||
| <li><a href="https://github.com/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/twitterstreamgenerator/TwitterConnection.java">Connection Management</a> | ||
| — The streaming pipeline example makes sure that only one Twitter connection is active at a time for a configuration. | ||
| </li> | ||
| <li><a href="https://github.com/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/twitterstreamgenerator/ReadFromTwitterDoFn.java">Terminating pipeline by time or elements</a> | ||
| — The streaming pipeline keeps track of time and data collecting so far and terminated when the limit specified in passed. | ||
| </li> | ||
| </ul> | ||
|
|
||
| ## Requirements | ||
|
|
||
| - Java 8 | ||
| - Twitter developer app account and streaming credentials. | ||
|
|
||
| This section describes what is needed to get the example up and running. | ||
|
|
||
| - Gradle preparation | ||
| - Local execution |
212 changes: 212 additions & 0 deletions
212
...in/java/org/apache/beam/examples/complete/twitterstreamgenerator/ReadFromTwitterDoFn.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,212 @@ | ||
| /* | ||
| * 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.beam.examples.complete.twitterstreamgenerator; | ||
|
|
||
| import java.io.IOException; | ||
| import java.io.Serializable; | ||
| import java.util.Objects; | ||
| import java.util.concurrent.BlockingQueue; | ||
| import org.apache.beam.sdk.coders.Coder; | ||
| import org.apache.beam.sdk.coders.SerializableCoder; | ||
| import org.apache.beam.sdk.io.range.OffsetRange; | ||
| import org.apache.beam.sdk.transforms.DoFn; | ||
| import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; | ||
| import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; | ||
| import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; | ||
| import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators; | ||
| import org.apache.beam.sdk.transforms.windowing.BoundedWindow; | ||
| import org.checkerframework.checker.nullness.qual.Nullable; | ||
| import org.joda.time.DateTime; | ||
| import org.joda.time.Duration; | ||
| import org.joda.time.Instant; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
| import twitter4j.Status; | ||
|
|
||
| /** Splittable dofn that read live data off twitter. * */ | ||
| @DoFn.UnboundedPerElement | ||
| final class ReadFromTwitterDoFn extends DoFn<TwitterConfig, String> { | ||
|
|
||
| private final DateTime startTime; | ||
|
|
||
| ReadFromTwitterDoFn() { | ||
| this.startTime = new DateTime(); | ||
| } | ||
| /* Logger for class.*/ | ||
| private static final Logger LOG = LoggerFactory.getLogger(ReadFromTwitterDoFn.class); | ||
|
|
||
| static class OffsetHolder implements Serializable { | ||
| public final @Nullable TwitterConfig twitterConfig; | ||
| public final @Nullable Long fetchedRecords; | ||
|
|
||
| OffsetHolder(@Nullable TwitterConfig twitterConfig, @Nullable Long fetchedRecords) { | ||
| this.twitterConfig = twitterConfig; | ||
| this.fetchedRecords = fetchedRecords; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(@Nullable Object o) { | ||
| if (this == o) { | ||
| return true; | ||
| } | ||
| if (o == null || getClass() != o.getClass()) { | ||
| return false; | ||
| } | ||
| OffsetHolder that = (OffsetHolder) o; | ||
| return Objects.equals(twitterConfig, that.twitterConfig) | ||
| && Objects.equals(fetchedRecords, that.fetchedRecords); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return Objects.hash(twitterConfig, fetchedRecords); | ||
| } | ||
| } | ||
|
|
||
| static class OffsetTracker extends RestrictionTracker<OffsetHolder, TwitterConfig> | ||
| implements Serializable { | ||
| private OffsetHolder restriction; | ||
| private final DateTime startTime; | ||
|
|
||
| OffsetTracker(OffsetHolder holder, DateTime startTime) { | ||
| this.restriction = holder; | ||
| this.startTime = startTime; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean tryClaim(TwitterConfig twitterConfig) { | ||
| LOG.debug( | ||
| "-------------- Claiming " | ||
| + twitterConfig.hashCode() | ||
| + " used to have: " | ||
| + restriction.fetchedRecords); | ||
| long fetchedRecords = | ||
| this.restriction == null || this.restriction.fetchedRecords == null | ||
| ? 0 | ||
| : this.restriction.fetchedRecords + 1; | ||
| long elapsedTime = System.currentTimeMillis() - startTime.getMillis(); | ||
| final long millis = 60 * 1000; | ||
| LOG.debug( | ||
| "-------------- Time running: {} / {}", | ||
| elapsedTime, | ||
| (twitterConfig.getMinutesToRun() * millis)); | ||
| if (fetchedRecords > twitterConfig.getTweetsCount() | ||
| || elapsedTime > twitterConfig.getMinutesToRun() * millis) { | ||
| return false; | ||
| } | ||
| this.restriction = new OffsetHolder(twitterConfig, fetchedRecords); | ||
| return true; | ||
| } | ||
|
|
||
| @Override | ||
| public OffsetHolder currentRestriction() { | ||
| return restriction; | ||
| } | ||
|
|
||
| @Override | ||
| public SplitResult<OffsetHolder> trySplit(double fractionOfRemainder) { | ||
| LOG.debug("-------------- Trying to split: fractionOfRemainder=" + fractionOfRemainder); | ||
| return SplitResult.of(new OffsetHolder(null, 0L), restriction); | ||
| } | ||
|
|
||
| @Override | ||
| public void checkDone() throws IllegalStateException {} | ||
|
|
||
| @Override | ||
| public IsBounded isBounded() { | ||
| return IsBounded.UNBOUNDED; | ||
| } | ||
| } | ||
|
|
||
| @GetInitialWatermarkEstimatorState | ||
| public Instant getInitialWatermarkEstimatorState(@Timestamp Instant currentElementTimestamp) { | ||
| return currentElementTimestamp; | ||
| } | ||
|
|
||
| private static Instant ensureTimestampWithinBounds(Instant timestamp) { | ||
| if (timestamp.isBefore(BoundedWindow.TIMESTAMP_MIN_VALUE)) { | ||
| timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; | ||
| } else if (timestamp.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE)) { | ||
| timestamp = BoundedWindow.TIMESTAMP_MAX_VALUE; | ||
| } | ||
| return timestamp; | ||
| } | ||
|
|
||
| @NewWatermarkEstimator | ||
| public WatermarkEstimators.Manual newWatermarkEstimator( | ||
| @WatermarkEstimatorState Instant watermarkEstimatorState) { | ||
| return new WatermarkEstimators.Manual(ensureTimestampWithinBounds(watermarkEstimatorState)); | ||
| } | ||
|
|
||
| @DoFn.GetInitialRestriction | ||
| public OffsetHolder getInitialRestriction(@Element TwitterConfig twitterConfig) | ||
| throws IOException { | ||
| return new OffsetHolder(null, 0L); | ||
| } | ||
|
|
||
| @DoFn.NewTracker | ||
| public RestrictionTracker<OffsetHolder, TwitterConfig> newTracker( | ||
| @Element TwitterConfig twitterConfig, @DoFn.Restriction OffsetHolder restriction) { | ||
| return new OffsetTracker(restriction, startTime); | ||
| } | ||
|
|
||
| @GetRestrictionCoder | ||
| public Coder<OffsetHolder> getRestrictionCoder() { | ||
| return SerializableCoder.of(OffsetHolder.class); | ||
| } | ||
|
|
||
| @DoFn.ProcessElement | ||
| public DoFn.ProcessContinuation processElement( | ||
| @Element TwitterConfig twitterConfig, | ||
| DoFn.OutputReceiver<String> out, | ||
| RestrictionTracker<OffsetRange, TwitterConfig> tracker, | ||
| ManualWatermarkEstimator<Instant> watermarkEstimator) { | ||
| LOG.debug("In Read From Twitter Do Fn"); | ||
| TwitterConnection twitterConnection = TwitterConnection.getInstance(twitterConfig); | ||
| BlockingQueue<Status> queue = twitterConnection.getQueue(); | ||
| if (queue.isEmpty()) { | ||
| if (checkIfDone(twitterConnection, twitterConfig, tracker)) { | ||
| return DoFn.ProcessContinuation.stop(); | ||
| } | ||
| } | ||
| while (!queue.isEmpty()) { | ||
| Status status = queue.poll(); | ||
| if (checkIfDone(twitterConnection, twitterConfig, tracker)) { | ||
| return DoFn.ProcessContinuation.stop(); | ||
| } | ||
| if (status != null) { | ||
| Instant currentInstant = Instant.ofEpochMilli(status.getCreatedAt().getTime()); | ||
| watermarkEstimator.setWatermark(currentInstant); | ||
| out.outputWithTimestamp(status.getText(), currentInstant); | ||
| } | ||
| } | ||
| return DoFn.ProcessContinuation.resume().withResumeDelay(Duration.standardSeconds(1)); | ||
| } | ||
|
|
||
| boolean checkIfDone( | ||
| TwitterConnection twitterConnection, | ||
| TwitterConfig twitterConfig, | ||
| RestrictionTracker<OffsetRange, TwitterConfig> tracker) { | ||
| if (!tracker.tryClaim(twitterConfig)) { | ||
| twitterConnection.closeStream(); | ||
| return true; | ||
| } else { | ||
| return false; | ||
| } | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.