[ 
https://issues.apache.org/jira/browse/FLINK-5697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16693525#comment-16693525
 ] 

ASF GitHub Bot commented on FLINK-5697:
---------------------------------------

mxm commented on a change in pull request #6980: [FLINK-5697] [kinesis] Add 
periodic per-shard watermark support
URL: https://github.com/apache/flink/pull/6980#discussion_r234705383
 
 

 ##########
 File path: 
flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
 ##########
 @@ -609,7 +667,115 @@ public int 
registerNewSubscribedShardState(KinesisStreamShardState newSubscribed
                                this.numberOfActiveShards.incrementAndGet();
                        }
 
-                       return subscribedShardsState.size() - 1;
+                       int shardStateIndex = subscribedShardsState.size() - 1;
+
+                       // track all discovered shards for watermark 
determination
+                       ShardWatermarkState sws = 
shardWatermarks.get(shardStateIndex);
+                       if (sws == null) {
+                               sws = new ShardWatermarkState();
+                               try {
+                                       sws.periodicWatermarkAssigner = 
InstantiationUtil.clone(periodicWatermarkAssigner);
+                               } catch (Exception e) {
+                                       throw new RuntimeException(e);
+                               }
+                               sws.lastUpdated = getCurrentTimeMillis();
+                               sws.lastRecordTimestamp = Long.MIN_VALUE;
+                               shardWatermarks.put(shardStateIndex, sws);
+                       }
+
+                       return shardStateIndex;
+               }
+       }
+
+       /**
+        * Return the current system time. Allow tests to override this to 
simulate progress for watermark
+        * logic.
+        *
+        * @return
+        */
+       @VisibleForTesting
+       protected long getCurrentTimeMillis() {
+               return System.currentTimeMillis();
+       }
+
+       /**
+        * Called periodically to emit a watermark. Checks all shards for the 
current event time
+        * watermark, and possibly emits the next watermark.
+        *
+        * <p>Shards that have not received an update for a certain interval 
are considered inactive so as
+        * to not hold back the watermark indefinitely. When all shards are 
inactive, the subtask will be
+        * marked as temporarily idle to not block downstream operators.
+        */
+       @VisibleForTesting
+       protected void emitWatermark() {
+               LOG.debug(
+                       "###evaluating watermark for subtask {} time {}",
+                       indexOfThisConsumerSubtask,
+                       getCurrentTimeMillis());
+               long potentialWatermark = Long.MAX_VALUE;
+               long idleTime =
+                       (shardIdleIntervalMillis > 0)
+                               ? getCurrentTimeMillis() - 
shardIdleIntervalMillis
+                               : Long.MAX_VALUE;
+
+               for (Map.Entry<Integer, ShardWatermarkState> e : 
shardWatermarks.entrySet()) {
+                       // consider only active shards, or those that would 
advance the watermark
+                       Watermark w = 
e.getValue().periodicWatermarkAssigner.getCurrentWatermark();
+                       if (w != null && (e.getValue().lastUpdated >= idleTime 
|| w.getTimestamp() > lastWatermark)) {
+                               potentialWatermark = 
Math.min(potentialWatermark, w.getTimestamp());
+                       }
+               }
+
+               // advance watermark if possible (watermarks can only be 
ascending)
+               if (potentialWatermark == Long.MAX_VALUE) {
 
 Review comment:
   Will this not be executed repeatedly? Maybe move this block inside the 
`potentialWatermark > lastWatermark`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Add per-shard watermarks for FlinkKinesisConsumer
> -------------------------------------------------
>
>                 Key: FLINK-5697
>                 URL: https://issues.apache.org/jira/browse/FLINK-5697
>             Project: Flink
>          Issue Type: New Feature
>          Components: Kinesis Connector, Streaming Connectors
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Thomas Weise
>            Priority: Major
>              Labels: pull-request-available
>
> It would be nice to let the Kinesis consumer be on-par in functionality with 
> the Kafka consumer, since they share very similar abstractions. Per-partition 
> / shard watermarks is something we can add also to the Kinesis consumer.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to