You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/04/11 05:27:36 UTC

[GitHub] [flink] blake-wilson opened a new pull request, #19417: [FLINK-20060][Connectors / Kinesis] Add a Collector to KinsesisDeserializationSchema

blake-wilson opened a new pull request, #19417:
URL: https://github.com/apache/flink/pull/19417

   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   https://issues.apache.org/jira/browse/FLINK-20060
   
   ## What is the purpose of the change
   
   Enables specifying a `DeserializationSchema` for the `KinesisConsumer` which uses a `Collector` to support cases where multiple values can be emitted by a single Kinesis record.
   
   The Jira issue description raised the worry about needing to hand messages inside collections across thread boundaries via a queue, but in this implementation the collections never traverse threads. There is some multi-threaded code which comes after deserialization, and I assume this is what the concern raised is referring to.
   
   ## Brief change log
   
     - The `KinesisDeserializationSchemaWrapper` inspects the provided `DeserializationSchema` to determine if a `Collector` is supported and uses it if so.
     - Updates `ShardConsumer` to enable deserializing multiple values per kinesis record via a `DeserializationSchema` using a `Collector`. 
   
   ## Verifying this change
   - [ ] This change is a trivial rework / code cleanup without any test coverage.
   - [ ] This change is already covered by existing tests.
   - [x] This change added tests and can be verified as follows:
     - Added unit test that `KinesisDeserializationSchemaWrapper` works for both schemas with and without `Collector`s
     - Manually verified the change by running a 4 node cluster with 1 JobManager and 4 TaskManagers, a stateful streaming program using `Collector` API. 
     -  Regression tested using the single-record only (non-`Collector`) `DeserializationSchema`
   
   Tested with a moderate (≈ 2MB/s) rate of data running for several hours
   
   ## Does this pull request potentially affect one of the following parts:
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: yes (`KinesisDeserializationSchema`)
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): yes
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? yes
     - If yes, how is the feature documented?
       - Feature is new to `KinesisDeserializationSchema` but common to `DeserializationSchema` in Flink. Removed code that treated the Kinesis connector's `DeserializationSchema` as a special case in not functioning with a `Collector`.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] dannycranmer commented on a diff in pull request #19417: [FLINK-20060][Connectors / Kinesis] Add a Collector to KinsesisDeserializationSchema

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on code in PR #19417:
URL: https://github.com/apache/flink/pull/19417#discussion_r866220480


##########
flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java:
##########
@@ -57,11 +58,9 @@ default void open(DeserializationSchema.InitializationContext context) throws Ex
      *     the record
      * @param stream the name of the Kinesis stream that this record was sent to
      * @param shardId The identifier of the shard the record was sent to
-     * @return the deserialized message as an Java object ({@code null} if the message cannot be
-     *     deserialized).
      * @throws IOException
      */
-    T deserialize(
+    List<T> deserialize(

Review Comment:
   This is a non-backwards compatible change. It will break all existing implementations of `KinesisDeserializationSchema`. I would rather introduce a new interface specifically for this usecase, then we can fork the code path based on the supplied deserialiser



##########
flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java:
##########
@@ -216,8 +217,13 @@ private void deserializeRecordForCollectionAndUpdateState(final UserRecord recor
                                 record.getSequenceNumber(), record.getSubSequenceNumber())
                         : new SequenceNumber(record.getSequenceNumber());
 
-        fetcherRef.emitRecordAndUpdateState(
-                value, approxArrivalTimestamp, subscribedShardStateIndex, collectedSequenceNumber);
+        for (T val : values) {
+            fetcherRef.emitRecordAndUpdateState(
+                    val,
+                    approxArrivalTimestamp,
+                    subscribedShardStateIndex,
+                    collectedSequenceNumber);
+        }
 

Review Comment:
   How about if values is empty? I think this would mean that the state would not be updated, and if the job restarts it would not restart from `collectedSequenceNumber`, it would use an earlier sequence number. While this is not going to impact processing semantics, it feels like a smell to me.  



##########
flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java:
##########
@@ -60,15 +64,26 @@ public void open(DeserializationSchema.InitializationContext context) throws Exc
     }
 
     @Override
-    public T deserialize(
+    public List<T> deserialize(
             byte[] recordValue,
             String partitionKey,
             String seqNum,
             long approxArrivalTimestamp,
             String stream,
             String shardId)
             throws IOException {
-        return deserializationSchema.deserialize(recordValue);
+
+        List<T> out = new ArrayList<>();
+        if (useCollector) {
+            ListCollector<T> coll = new ListCollector<>(out);
+            deserializationSchema.deserialize(recordValue, coll);
+        } else {

Review Comment:
   There are no unit tests to cover this code path, can you please add



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #19417: [FLINK-20060][Connectors / Kinesis] Add a Collector to KinsesisDeserializationSchema

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #19417:
URL: https://github.com/apache/flink/pull/19417#issuecomment-1094569830

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "90ae89ff5e1d9e66e0a1e898da3fc85ad50cf04e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "90ae89ff5e1d9e66e0a1e898da3fc85ad50cf04e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 90ae89ff5e1d9e66e0a1e898da3fc85ad50cf04e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] blake-wilson commented on pull request #19417: [FLINK-20060][Connectors / Kinesis] Add a Collector to KinsesisDeserializationSchema

Posted by GitBox <gi...@apache.org>.
blake-wilson commented on PR #19417:
URL: https://github.com/apache/flink/pull/19417#issuecomment-1095295771

   Test failure is from a Kafka integration test and appears to be unrelated
   ```
   2022-04-11T07:53:52.3560140Z Apr 11 07:53:52 Caused by: java.lang.IllegalStateException: Cannot mark for checkpoint 42, already marked for checkpoint 41
   2022-04-11T07:53:52.3560857Z Apr 11 07:53:52 	at org.apache.flink.runtime.operators.coordination.OperatorEventValve.markForCheckpoint(OperatorEventValve.java:113)
   ```


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org