diff --git a/README.md b/README.md
index 029e4eb..e77ade6 100644
--- a/README.md
+++ b/README.md
@@ -16,7 +16,7 @@ The Amazon Kinesis Storm spout fetches data records from Amazon Kinesis and emit
The Amazon Kinesis Storm spout can be configured to retry failed records. By default, it retries a failed record 3 times. If a record fails and the retry limit has been reached, the spout will log an error and skip over the record. The spout buffers pending records in memory, so it can re-emit a failed record without having to re-fetch the record from Amazon Kinesis. The spout sets the checkpoint to the highest sequence number that has been ack'ed (or exhausted retry attempts).
-To use the spout, you'll need to add it to your Storm topology.
+To use the spout, you'll need to add it to your Storm topology.
+ **KinesisSpout**: Constructs an instance of the spout, using your AWS credentials and the configuration specified in KinesisSpoutConfig (as well as com.amazonaws.ClientConfiguration, via the AWS SDK). Each task executed by the spout operates on a distinct set of Amazon Kinesis shards. Shard states are periodically committed to ZooKeeper. When the spout is deactivated, it will disconnect from ZooKeeper, but the spout will continue monitoring its local state so you can activate it again later.
+ **KinesisSpoutConfig**: Configures the spout, including the Storm topology name, the Amazon Kinesis stream name, the endpoint for connecting to ZooKeeper, and the prefix for the ZooKeeper paths where the spout state is stored. See the samples folder for configuration examples.
@@ -28,7 +28,7 @@ The samples folder includes a sample topology and sample bolt, using the number
1. Edit the *.properties file to configure your Storm topology, Amazon Kinesis stream, and ZooKeeper details. For your AWS Credentials, we recommend using IAM roles on Amazon EC2 when possible. You can also specify your credentials using system properties, environment variables, or AwsCredentials.properties.
2. Package the spout and the sample (including all dependencies but excluding Storm itself) into one JAR file.
-3. Deploy the package to Storm via the JAR file, e.g., `storm jar my-spout-sample.jar SampleTopology sample.properties RemoteMode`
+3. Deploy the package to Storm via the JAR file, e.g., `storm jar my-spout-sample.jar SampleTopology sample.properties RemoteMode`
## Release Notes
### Release 1.1 (October 21, 2014)
@@ -36,10 +36,6 @@ The samples folder includes a sample topology and sample bolt, using the number
+ Added region name support
-### Future Work
-
-+ Handle closed, split, and merged shards
-
## Related Resources
[Amazon Kinesis Developer Guide](http://docs.aws.amazon.com/kinesis/latest/dev/introduction.html)
diff --git a/pom.xml b/pom.xml
index 5caad43..164a39d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -6,7 +6,7 @@
kinesis-storm-spoutjarAmazon Kinesis Storm Spout for Java
- 1.1.0
+ 1.2.0 The Amazon Kinesis Storm Spout helps Java developers integrate Amazon Kinesis with Storm.https://aws.amazon.com/kinesis
@@ -30,8 +30,8 @@
13.0-final3.0
-
-
+
+
com.amazonaws
diff --git a/src/main/java/com/amazonaws/services/kinesis/stormspout/BufferedGetter.java b/src/main/java/com/amazonaws/services/kinesis/stormspout/BufferedGetter.java
index 86c1708..910715e 100644
--- a/src/main/java/com/amazonaws/services/kinesis/stormspout/BufferedGetter.java
+++ b/src/main/java/com/amazonaws/services/kinesis/stormspout/BufferedGetter.java
@@ -72,7 +72,7 @@ public Records getNext(int maxNumberOfRecords) {
ensureBuffered();
if (!it.hasNext() && buffer.isEndOfShard()) {
- return new Records(ImmutableList. of(), true);
+ return new Records(ImmutableList. of(), true, buffer.isReshard());
}
ImmutableList.Builder recs = new ImmutableList.Builder<>();
@@ -94,7 +94,7 @@ public Records getNext(int maxNumberOfRecords) {
}
}
- return new Records(recs.build(), false);
+ return new Records(recs.build(), false, buffer.isReshard());
}
@Override
diff --git a/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisHelper.java b/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisHelper.java
index 14f133d..10826ae 100644
--- a/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisHelper.java
+++ b/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisHelper.java
@@ -151,7 +151,8 @@ private String addTruncatedShardList(final Map spoutShards, f
for (Shard s : streamShards) {
currShard = s.getShardId();
- spoutShards.put(s.getShardId(), new ShardInfo(s.getShardId()));
+ final boolean open = s.getSequenceNumberRange().getEndingSequenceNumber() == null;
+ spoutShards.put(s.getShardId(), new ShardInfo(s.getShardId(), open));
if (s.getParentShardId() != null && s.getAdjacentParentShardId() != null) {
// It's a merge. Set both parents of the merge to merge into this shard.
diff --git a/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisShardGetter.java b/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisShardGetter.java
index 48f2988..75a11ef 100644
--- a/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisShardGetter.java
+++ b/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisShardGetter.java
@@ -49,7 +49,7 @@ class KinesisShardGetter implements IShardGetter {
private static final long BACKOFF_MILLIS = 500L;
private final String streamName;
- private final String shardId;
+ private final ShardInfo shard;
private final AmazonKinesisClient kinesisClient;
private String shardIterator;
@@ -57,12 +57,12 @@ class KinesisShardGetter implements IShardGetter {
/**
* @param streamName Name of the Kinesis stream
- * @param shardId Fetch data from this shard
+ * @param shard Fetch data from this shard
* @param kinesisClient Kinesis client to use when making requests.
*/
- KinesisShardGetter(final String streamName, final String shardId, final AmazonKinesisClient kinesisClient) {
+ KinesisShardGetter(final String streamName, final ShardInfo shard, final AmazonKinesisClient kinesisClient) {
this.streamName = streamName;
- this.shardId = shardId;
+ this.shard = shard;
this.kinesisClient = kinesisClient;
this.shardIterator = "";
this.positionInShard = ShardPosition.end();
@@ -72,12 +72,12 @@ class KinesisShardGetter implements IShardGetter {
public Records getNext(int maxNumberOfRecords)
throws AmazonClientException, ResourceNotFoundException, InvalidArgumentException {
if (shardIterator == null) {
- LOG.debug(this + " Null shardIterator for " + shardId + ". This can happen if shard is closed.");
+ LOG.debug(this + " Null shardIterator for " + shard.getShardId() + ". This can happen if shard is closed.");
return Records.empty(true);
}
- final ImmutableList.Builder records = new ImmutableList.Builder<>();
-
+ final ImmutableList.Builder records = new ImmutableList.Builder();
+
try {
final GetRecordsRequest request = new GetRecordsRequest();
request.setShardIterator(shardIterator);
@@ -93,20 +93,22 @@ public Records getNext(int maxNumberOfRecords)
+ maxNumberOfRecords + ").");
}
- shardIterator = result.getNextShardIterator();
+ shardIterator = result.getNextShardIterator();
} catch (AmazonClientException e) {
// We'll treat this equivalent to fetching 0 records - the spout drives the retry as part of nextTuple()
// We don't sleep here - we can continue processing ack/fail on the spout thread.
- LOG.error(this + "Caught exception when fetching records for " + shardId, e);
+ LOG.error(this + "Caught exception when fetching records for " + shard.getShardId(), e);
}
- return new Records(records.build(), shardIterator == null);
+ final boolean endOfShard = shardIterator == null;
+ final boolean reshard = endOfShard && shard.getShardOpen();
+ return new Records(records.build(), endOfShard, reshard);
}
@Override
public void seek(ShardPosition position)
throws AmazonClientException, ResourceNotFoundException, InvalidSeekPositionException {
- LOG.info("Seeking to " + position);
+ LOG.info(this + " Seeking to " + position + " for shard " + shard.getShardId());
ShardIteratorType iteratorType;
String seqNum = null;
@@ -145,12 +147,12 @@ public void seek(ShardPosition position)
@Override
public String getAssociatedShard() {
- return shardId;
+ return shard.getShardId();
}
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE).append("shardId", shardId).toString();
+ return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE).append("shardId", shard.getShardId()).toString();
}
private String seek(final ShardIteratorType iteratorType, final String seqNum)
@@ -158,7 +160,7 @@ private String seek(final ShardIteratorType iteratorType, final String seqNum)
final GetShardIteratorRequest request = new GetShardIteratorRequest();
request.setStreamName(streamName);
- request.setShardId(shardId);
+ request.setShardId(shard.getShardId());
request.setShardIteratorType(iteratorType);
// SeqNum is only set on {AT, AFTER}_SEQUENCE_NUMBER, so this is safe.
diff --git a/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisShardGetterBuilder.java b/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisShardGetterBuilder.java
index c289644..6f63450 100644
--- a/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisShardGetterBuilder.java
+++ b/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisShardGetterBuilder.java
@@ -16,6 +16,7 @@
package com.amazonaws.services.kinesis.stormspout;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSortedMap;
/**
* Builds KinesisShardGetters.
@@ -31,7 +32,7 @@ class KinesisShardGetterBuilder implements IShardGetterBuilder {
/**
* Constructor.
- *
+ *
* @param streamName Kinesis stream to create the getters in.
* @param helper Used to get the AmazonKinesisClient object (used by the getters).
*/
@@ -47,12 +48,15 @@ public KinesisShardGetterBuilder(final String streamName,
@Override
public ImmutableList buildGetters(ImmutableList shardAssignment) {
- ImmutableList.Builder builder = new ImmutableList.Builder<>();
+ ImmutableList.Builder builder = new ImmutableList.Builder();
+ ImmutableSortedMap shards = helper.getShardList();
- for (String shard : shardAssignment) {
- builder.add(new BufferedGetter(new KinesisShardGetter(streamName, shard, helper.getSharedkinesisClient()),
- maxRecordsPerCall,
- emptyRecordListBackoffMillis));
+ for (String shardId : shardAssignment) {
+ KinesisShardGetter getter = new KinesisShardGetter(
+ streamName,
+ shards.get(shardId),
+ helper.getSharedkinesisClient());
+ builder.add(new BufferedGetter(getter, maxRecordsPerCall, emptyRecordListBackoffMillis));
}
return builder.build();
diff --git a/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisSpout.java b/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisSpout.java
index 7130026..066f3da 100644
--- a/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisSpout.java
+++ b/src/main/java/com/amazonaws/services/kinesis/stormspout/KinesisSpout.java
@@ -36,35 +36,44 @@
import com.amazonaws.services.kinesis.stormspout.state.IKinesisSpoutStateManager;
import com.amazonaws.services.kinesis.stormspout.state.zookeeper.ZookeeperStateManager;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSortedMap;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
/**
* Storm spout for Amazon Kinesis. The spout fetches data from Kinesis and emits a tuple for each data record.
- *
+ *
* Note: every spout task handles a distinct set of shards.
*/
public class KinesisSpout implements IRichSpout, Serializable {
private static final long serialVersionUID = 7707829996758189836L;
private static final Logger LOG = LoggerFactory.getLogger(KinesisSpout.class);
- private final InitialPositionInStream initialPosition;
-
// Initialized before open
+ private final InitialPositionInStream initialPosition;
+ private final KinesisHelper shardListGetter;
private final KinesisSpoutConfig config;
- private final IShardListGetter shardListGetter;
+ private final long emptyRecordListSleepTimeMillis = 5L;
+ private final DeadLetterQueue deadLetterQueue;
private final IShardGetterBuilder getterBuilder;
- private long emptyRecordListSleepTimeMillis = 5L;
// Initialized on open
private transient SpoutOutputCollector collector;
private transient TopologyContext context;
- private transient IKinesisSpoutStateManager stateManager;
+ private transient ZookeeperStateManager stateManager;
private transient long lastCommitTime;
/**
* Constructs an instance of the spout with just enough data to bootstrap the state from.
* Construction done here is common to all spout tasks, whereas the IKinesisSpoutStateManager created
* in activate() is task specific.
- *
+ *
* @param config Spout configuration.
* @param credentialsProvider Used when making requests to Kinesis.
* @param clientConfiguration Client configuration used when making calls to Kinesis.
@@ -86,20 +95,6 @@ public KinesisSpout(KinesisSpoutConfig config,
this.initialPosition = config.getInitialPositionInStream();
}
- /**
- * @param config Spout configuration.
- * @param shardListGetter Used to list the shards in the stream.
- * @param getterBuilder Used for creating shard getters for a task.
- */
- KinesisSpout(final KinesisSpoutConfig config,
- final IShardListGetter shardListGetter,
- final IShardGetterBuilder getterBuilder) {
- this.config = config;
- this.shardListGetter = shardListGetter;
- this.getterBuilder = getterBuilder;
- this.initialPosition = config.getInitialPositionInStream();
- }
-
@Override
public void open(@SuppressWarnings("rawtypes") final Map conf,
final TopologyContext spoutContext,
@@ -162,7 +157,7 @@ public void nextTuple() {
String currentShardId = getter.getAssociatedShard();
Record rec = null;
boolean isRetry = false;
-
+
if (stateManager.shouldRetry(currentShardId)) {
rec = stateManager.recordToRetry(currentShardId);
if (LOG.isDebugEnabled()) {
@@ -171,9 +166,14 @@ public void nextTuple() {
}
isRetry = true;
} else {
- final ImmutableList records = getter.getNext(1).getRecords();
- if ((records != null) && (!records.isEmpty())) {
- rec = records.get(0);
+ final Records records = getter.getNext(1);
+ final ImmutableList recordList = records.getRecords();
+ if ((recordList != null) && (!recordList.isEmpty())) {
+ rec = recordList.get(0);
+ }
+ if (records.isReshard()) {
+ LOG.info(this + " detected reshard event for shard " + currentShardId);
+ stateManager.handleReshard();
}
}
@@ -181,11 +181,8 @@ public void nextTuple() {
// Copy record (ByteBuffer.duplicate()) so bolts in the same JVM don't affect the object (e.g. retries)
Record recordToEmit = copyRecord(rec);
List