[
https://issues.apache.org/jira/browse/STORM-1839?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15402724#comment-15402724
]
ASF GitHub Bot commented on STORM-1839:
---------------------------------------
Github user harshach commented on a diff in the pull request:
https://github.com/apache/storm/pull/1586#discussion_r73043179
--- Diff:
external/storm-kinesis/src/main/java/org/apache/storm/kinesis/spout/KinesisRecordsManager.java
---
@@ -0,0 +1,449 @@
+/**
+ * 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.storm.kinesis.spout;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import
com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException;
+import com.amazonaws.services.kinesis.model.Record;
+import com.amazonaws.services.kinesis.model.Shard;
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeSet;
+
+class KinesisRecordsManager {
+ private static final Logger LOG =
LoggerFactory.getLogger(KinesisRecordsManager.class);
+ // object handling zk interaction
+ private transient ZKConnection zkConnection;
+ // object handling interaction with kinesis
+ private transient KinesisConnection kinesisConnection;
+ // Kinesis Spout KinesisConfig object
+ private transient final KinesisConfig kinesisConfig;
+ // Queue of records per shard fetched from kinesis and are waiting to
be emitted
+ private transient Map<String, LinkedList<Record>> toEmitPerShard = new
HashMap<>();
+ // Map of records that were fetched from kinesis as a result of
failure and are waiting to be emitted
+ private transient Map<KinesisMessageId, Record>
failedandFetchedRecords = new HashMap<>();
+ // Sequence numbers per shard that have been emitted. LinkedHashSet as
we need to remove on ack or fail. At the same time order is needed to figure
out the
+ // sequence number to commit. Logic explained in commit
+ private transient Map<String, TreeSet<BigInteger>> emittedPerShard =
new HashMap<>();
+ // sorted acked sequence numbers - needed to figure out what sequence
number can be committed
+ private transient Map<String, TreeSet<BigInteger>> ackedPerShard = new
HashMap<>();
+ // sorted failed sequence numbers - needed to figure out what sequence
number can be committed
+ private transient Map<String, TreeSet<BigInteger>> failedPerShard =
new HashMap<>();
+ // shard iterator corresponding to position in shard for new messages
+ private transient Map<String, String> shardIteratorPerShard = new
HashMap<>();
+ // last fetched sequence number corresponding to position in shard
+ private transient Map<String, String> fetchedSequenceNumberPerShard =
new HashMap<>();
+ // shard iterator corresponding to position in shard for failed
messages
+ private transient Map<KinesisMessageId, String>
shardIteratorPerFailedMessage = new HashMap<>();
+ // timestamp to decide when to commit to zk again
+ private transient long lastCommitTime;
+ // boolean to track deactivated state
+ private transient boolean deactivated;
+
+ KinesisRecordsManager (KinesisConfig kinesisConfig) {
+ this.kinesisConfig = kinesisConfig;
+ this.zkConnection = new ZKConnection(kinesisConfig.getZkInfo());
+ this.kinesisConnection = new
KinesisConnection(kinesisConfig.getKinesisConnectionInfo());
+ }
+
+ void initialize (int myTaskIndex, int totalTasks) {
+ deactivated = false;
+ lastCommitTime = System.currentTimeMillis();
+ kinesisConnection.initialize();
+ zkConnection.initialize();
+ List<Shard> shards =
kinesisConnection.getShardsForStream(kinesisConfig.getStreamName());
+ LOG.info("myTaskIndex is " + myTaskIndex);
+ LOG.info("totalTasks is " + totalTasks);
+ int i = myTaskIndex;
+ while (i < shards.size()) {
+ LOG.info("Shard id " + shards.get(i).getShardId() + " assigned
to task " + myTaskIndex);
+ toEmitPerShard.put(shards.get(i).getShardId(), new
LinkedList<Record>());
+ i += totalTasks;
+ }
+ initializeFetchedSequenceNumbers();
+ refreshShardIteratorsForNewRecords();
+ }
+
+ void next (SpoutOutputCollector collector) {
+ if (shouldCommit()) {
+ commit();
+ }
+ KinesisMessageId failedMessageId =
kinesisConfig.getFailedMessageRetryHandler().getNextFailedMessageToRetry();
+ if (failedMessageId != null) {
+ // if the retry service returns a message that is not in
failed set then ignore it. should never happen
+ BigInteger failedSequenceNumber = new
BigInteger(failedMessageId.getSequenceNumber());
+ if (failedPerShard.containsKey(failedMessageId.getShardId())
&&
failedPerShard.get(failedMessageId.getShardId()).contains(failedSequenceNumber))
{
+ if (!failedandFetchedRecords.containsKey(failedMessageId))
{
+ fetchFailedRecords(failedMessageId);
+ }
+ if (emitFailedRecord(collector, failedMessageId)) {
+
failedPerShard.get(failedMessageId.getShardId()).remove(failedSequenceNumber);
+
kinesisConfig.getFailedMessageRetryHandler().failedMessageEmitted(failedMessageId);
+ return;
+ } else {
+ LOG.warn("failedMessageEmitted not called on retrier
for " + failedMessageId + ". This can happen a few times but should not happen
" +
+ "infinitely");
+ }
+ } else {
+ LOG.warn("failedPerShard does not contain " +
failedMessageId + ". This should never happen.");
+ }
+ }
+ LOG.debug("No failed record to emit for now. Hence will try to
emit new records");
+ // if maximum uncommitted records count has reached, so dont emit
any new records and return
+ if (!(getUncommittedRecordsCount() <
kinesisConfig.getMaxUncommittedRecords())) {
+ LOG.warn("maximum uncommitted records count has reached. so
not emitting any new records and returning");
+ return;
+ }
+ // early return as no shard is assigned - probably because number
of executors > number of shards
+ if (toEmitPerShard.isEmpty()) {
+ LOG.warn("No shard is assigned to this task. Hence not
emitting any tuple.");
+ return;
+ }
+
+ if (shouldFetchNewRecords()) {
+ fetchNewRecords();
+ }
+ emitNewRecord(collector);
+ }
+
+ void ack (KinesisMessageId kinesisMessageId) {
+ // for an acked message add it to acked set and remove it from
emitted and failed
+ String shardId = kinesisMessageId.getShardId();
+ BigInteger sequenceNumber = new
BigInteger(kinesisMessageId.getSequenceNumber());
+ LOG.debug("Ack received for shardId: " + shardId + "
sequenceNumber: " + sequenceNumber);
+ // if an ack is received for a message then add it to the
ackedPerShard TreeSet. TreeSet because while committing we need to figure out
what is the
+ // highest sequence number that can be committed for this shard
+ if (!ackedPerShard.containsKey(shardId)) {
+ ackedPerShard.put(shardId, new TreeSet<BigInteger>());
+ }
+ ackedPerShard.get(shardId).add(sequenceNumber);
+ // if the acked message was in emittedPerShard that means we need
to remove it from the emittedPerShard(which keeps track of in flight tuples)
+ if (emittedPerShard.containsKey(shardId)) {
+ TreeSet<BigInteger> emitted = emittedPerShard.get(shardId);
+ emitted.remove(sequenceNumber);
+ }
+ // an acked message should not be in failed since if it fails and
gets re-emitted it moves to emittedPerShard from failedPerShard. Defensive
coding.
+ // Remove it from failedPerShard anyway
+ if (failedPerShard.containsKey(shardId)) {
+ failedPerShard.get(shardId).remove(sequenceNumber);
+ }
+ // if an ack is for a message that failed once at least and was
re-emitted then the record itself will be in failedAndFetchedRecords. We use
that to
+ // determine if the FailedMessageRetryHandler needs to be told
about it and then remove the record itself to clean up memory
+ if (failedandFetchedRecords.containsKey(kinesisMessageId)) {
+
kinesisConfig.getFailedMessageRetryHandler().acked(kinesisMessageId);
+ failedandFetchedRecords.remove(kinesisMessageId);
+ }
+ // keep committing when topology is deactivated since ack and fail
keep getting called on deactivated topology
+ if (deactivated) {
+ commit();
+ }
+ }
+
+ void fail (KinesisMessageId kinesisMessageId) {
+ String shardId = kinesisMessageId.getShardId();
+ BigInteger sequenceNumber = new
BigInteger(kinesisMessageId.getSequenceNumber());
+ LOG.debug("Fail received for shardId: " + shardId + "
sequenceNumber: " + sequenceNumber);
+ // for a failed message add it to failed set if it will be
retried, otherwise ack it; remove from emitted either way
+ if
(kinesisConfig.getFailedMessageRetryHandler().failed(kinesisMessageId)) {
+ if (!failedPerShard.containsKey(shardId)) {
+ failedPerShard.put(shardId, new TreeSet<BigInteger>());
+ }
+ failedPerShard.get(shardId).add(sequenceNumber);
+ TreeSet<BigInteger> emitted = emittedPerShard.get(shardId);
+ emitted.remove(sequenceNumber);
+ } else {
+ ack(kinesisMessageId);
+ }
+ // keep committing when topology is deactivated since ack and fail
keep getting called on deactivated topology
+ if (deactivated) {
+ commit();
+ }
+ }
+
+ void commit () {
+ // We have three mutually disjoint treesets per shard at any given
time to keep track of what sequence number can be committed to zookeeper.
+ // emittedPerShard, ackedPerShard and failedPerShard. Any record
starts by entering emittedPerShard. On ack it moves from emittedPerShard to
+ // ackedPerShard and on fail if retry service tells us to retry
then it moves from emittedPerShard to failedPerShard. The failed records will
move from
+ // failedPerShard to emittedPerShard when the failed record is
emitted again as a retry.
+ // Logic for deciding what sequence number to commit is find the
highest sequence number from ackedPerShard called X such that there is no
sequence
+ // number Y in emittedPerShard or failedPerShard that satisfies X
> Y. For e.g. if ackedPerShard is 1,4,5, emittedPerShard is 2,6 and
+ // failedPerShard is 3,7 then we can only commit 1 and not 4
because 2 is still pending and 3 has failed
+ for (String shardId: toEmitPerShard.keySet()) {
+ if (ackedPerShard.containsKey(shardId)) {
+ BigInteger commitSequenceNumberBound = null;
+ if (failedPerShard.containsKey(shardId) &&
!failedPerShard.get(shardId).isEmpty()) {
+ commitSequenceNumberBound =
failedPerShard.get(shardId).first();
+ }
+ if (emittedPerShard.containsKey(shardId) &&
!emittedPerShard.get(shardId).isEmpty()) {
+ BigInteger smallestEmittedSequenceNumber =
emittedPerShard.get(shardId).first();
+ if (commitSequenceNumberBound == null ||
(commitSequenceNumberBound.compareTo(smallestEmittedSequenceNumber) == 1)) {
+ commitSequenceNumberBound =
smallestEmittedSequenceNumber;
+ }
+ }
+ Iterator<BigInteger> ackedSequenceNumbers =
ackedPerShard.get(shardId).iterator();
+ BigInteger ackedSequenceNumberToCommit = null;
+ while (ackedSequenceNumbers.hasNext()) {
+ BigInteger ackedSequenceNumber =
ackedSequenceNumbers.next();
+ if (commitSequenceNumberBound == null ||
(commitSequenceNumberBound.compareTo(ackedSequenceNumber) == 1)) {
+ ackedSequenceNumberToCommit = ackedSequenceNumber;
+ ackedSequenceNumbers.remove();
+ } else {
+ break;
+ }
+ }
+ if (ackedSequenceNumberToCommit != null) {
+ Map<Object, Object> state = new HashMap<>();
+ state.put("committedSequenceNumber",
ackedSequenceNumberToCommit.toString());
+ LOG.debug("Committing sequence number " +
ackedSequenceNumberToCommit.toString() + " for shardId " + shardId);
+
zkConnection.commitState(kinesisConfig.getStreamName(), shardId, state);
+ }
+ }
+ }
+ lastCommitTime = System.currentTimeMillis();
+ }
+
+ void activate () {
+ LOG.info("Activate called");
+ deactivated = false;
+ kinesisConnection.initialize();
+ }
+
+ void deactivate () {
+ LOG.info("Deactivate called");
+ deactivated = true;
+ commit();
+ kinesisConnection.shutdown();
+ }
+
+ void close () {
+ commit();
+ kinesisConnection.shutdown();
+ zkConnection.shutdown();
+ }
+
+ // fetch records from kinesis starting at sequence number for message
passed as argument. Any other messages fetched and are in the failed queue will
also
+ // be kept in memory to avoid going to kinesis again for retry
+ private void fetchFailedRecords (KinesisMessageId kinesisMessageId) {
+ // if shard iterator not present for this message, get it
+ if (!shardIteratorPerFailedMessage.containsKey(kinesisMessageId)) {
+ refreshShardIteratorForFailedRecord(kinesisMessageId);
+ }
+ String shardIterator =
shardIteratorPerFailedMessage.get(kinesisMessageId);
+ LOG.debug("Fetching failed records for shard id :" +
kinesisMessageId.getShardId() + " at sequence number " +
kinesisMessageId.getSequenceNumber() +
+ " using shardIterator " + shardIterator);
+ try {
+ GetRecordsResult getRecordsResult =
kinesisConnection.fetchRecords(shardIterator);
+ if (getRecordsResult != null) {
+ List<Record> records = getRecordsResult.getRecords();
+ LOG.debug("Records size from fetchFailedRecords is " +
records.size());
+ // update the shard iterator to next one in case this
fetch does not give the message.
+ shardIteratorPerFailedMessage.put(kinesisMessageId,
getRecordsResult.getNextShardIterator());
+ if (records.size() == 0) {
+ LOG.warn("No records returned from kinesis. Hence
sleeping for 1 second");
+ Thread.sleep(1000);
--- End diff --
Sorry missed these earlier. Any reason for it to sleep exactly one sec or
should it be configurable.
> Kinesis Spout
> -------------
>
> Key: STORM-1839
> URL: https://issues.apache.org/jira/browse/STORM-1839
> Project: Apache Storm
> Issue Type: Improvement
> Reporter: Sriharsha Chintalapani
> Assignee: Priyank Shah
>
> As Storm is increasingly used in Cloud environments. It will great to have a
> Kinesis Spout integration in Apache Storm.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)