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 2020/05/14 15:49:30 UTC

[GitHub] [flink] freezhan opened a new pull request #12157: fix: kafka producer transactional.id too long

freezhan opened a new pull request #12157:
URL: https://github.com/apache/flink/pull/12157


   [FLINK-17691](https://issues.apache.org/jira/browse/FLINK-17691)
   
   # What is the purpose of the change
   
   This pull request is to fix the bug that the kafka producer011 failed to send msg when the auto setting transactional.id too long
   
   When in `Semantic.EXACTLY_ONCE`  mode, the producer011 will ignore the transactional.id which user defined, But auto generator by [TransactionalIdsGenerator](https://github.com/apache/flink/blob/ff861c2d8d522638821708fbefaea0bfdc77d358/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/TransactionalIdsGenerator.java#L47)
   
   # Brief change log
   
   * Do md5 on the transactional.id prefix
   
   # Verifying this change
   
   * create a sink kafka job in `Semantic.EXACTLY_ONCE` mode
   * make the operatorName's length large than 32767 (or a large sql operator, which sql string's length is larger than 32767)
    [FlinkKafkaProducer011 init TransactionalIdsGenerator](https://github.com/apache/flink/blob/ff861c2d8d522638821708fbefaea0bfdc77d358/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java#L839) prefix
     ```java
      getRuntimeContext().getTaskName() + "-" + ((StreamingRuntimeContext) getRuntimeContext()).getOperatorUniqueID()
     ```
   
   # 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): (no)
   * The serializers: (no)
   * The runtime per-record code paths (performance sensitive): (no)
   * Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (no)
   * The S3 file system connector: (no)
   
   # Documentation
   * Does this pull request introduce a new feature? (no)
   * If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 9fa966f648aa7d97e6000938970039e83b8e73f2 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/166258938) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 4d9d19dd8a1bcaa317dc8f0550186bba13f7c103 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166056411) Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326) 
   * 190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/166161943) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] freezhan commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
freezhan commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-633529964


   @flinkbot @rmetzger 
   The Azure Failed message is
   [error log](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_apis/build/builds/2121/logs/70)
   
   ```shell
   2020-05-25T08:34:39.2315111Z [ERROR] Tests run: 2, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 32.345 s <<< FAILURE! - in org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerMigrationOperatorTest
   2020-05-25T08:34:39.2316504Z [ERROR] testRestoreProducer[Migration Savepoint: 1.9](org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerMigrationOperatorTest)  Time elapsed: 3.051 s  <<< ERROR!
   2020-05-25T08:34:39.2317470Z org.apache.kafka.common.KafkaException: Socket server failed to bind to 0.0.0.0:36820: Address already in use.
   2020-05-25T08:34:39.2318339Z 	at kafka.network.Acceptor.openServerSocket(SocketServer.scala:573)
   ```
   
   My change does not involve any port-related code.
   Please help to check whether it is the master branch that has this problem. 
   
   In addition, is there any documentation on how to run CICD locally


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121",
       "triggerID" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4791",
       "triggerID" : "635556812",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121",
       "triggerID" : "635556812",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 45572c3aec3b7ea7664c53e2a9b17cff2912ba24 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4791) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 409246b65de6c5e76e6462ed4a55d73c7dc76c22 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166181069) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 07dcda24cfb9baa75b14b5ec953eabb9603e3082 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166205499) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384) 
   * b097f82badd3f5148a87d03842d8ee1ce3577fc6 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/166230440) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] freezhan commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
freezhan commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-656480710


   @aljoscha  How about letting the user pass this prefix parameter, then then also add a check inside the generator to verify that we really obey the length cap.
   
   Transaction.id should have a business meaning.
   
   I don't think the following lengthy context makes any sense, it's not readable if it's too long, right
   ```java
    getRuntimeContext().getTaskName() + "-" + ((StreamingRuntimeContext) getRuntimeContext()).getOperatorUniqueID()
   ```
   
   


----------------------------------------------------------------
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.

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



[GitHub] [flink] freezhan commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
freezhan commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-659332375


   > Ahh, this discussion now reminded me of this older issue: https://issues.apache.org/jira/browse/FLINK-11654. Can you ask Jiangjie Qin there if he's still working on the issue? If not maybe you can take over.
   
   Thank you for your reply, I agree with the conclusion of that issue. It's very thoughtful.
   
   1. Expose transactionIdPrefix (**And add length checks**)
   2. Fail Fast when overriding Kafka properties
   
   


----------------------------------------------------------------
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.

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



[GitHub] [flink] freezhan commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
freezhan commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-656483534


   In addition, our internal temporary repair method is
   
   ```java
   		this.prefix = prefix.substring(0, Math.min(prefix.length(), 200)) + "-" + EncodingUtils.hex(EncodingUtils.md5(prefix));
   ```
   how about that? 
   
   Can add this sentence before calling the `TransactionalIdsGenerator ` constructor, and then add a length check inside the constructor.
   
   But I still think it makes sense to let users pass it in.
   
   If the user does not specify the prefix parameter, the default is a whole bunch of context characters(Keep it the same as before)


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 4d9d19dd8a1bcaa317dc8f0550186bba13f7c103 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166056411) Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 4d9d19dd8a1bcaa317dc8f0550186bba13f7c103 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166056411) Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326) 
   * 190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 07dcda24cfb9baa75b14b5ec953eabb9603e3082 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166205499) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384) 
   * b097f82badd3f5148a87d03842d8ee1ce3577fc6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121",
       "triggerID" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4791",
       "triggerID" : "635556812",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121",
       "triggerID" : "635556812",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 45572c3aec3b7ea7664c53e2a9b17cff2912ba24 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4791) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 4d9d19dd8a1bcaa317dc8f0550186bba13f7c103 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] rmetzger commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-634875003


   The easiest way to re-trigger an Azure run is to rebase to push again (for example after a rebase).
   
   You can run the tests locally by running "mvn clean verify" (you can do that also in the Kafka dir to run only the Kafka tests).


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121",
       "triggerID" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4791",
       "triggerID" : "635556812",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 45572c3aec3b7ea7664c53e2a9b17cff2912ba24 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4791) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166161943) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350) 
   * f0bdd24a7d955fb17aadc7c022ab2bd06461f869 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot commented on pull request #12157: fix: kafka producer transactional.id too long

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


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit c72006072a887727d7ac814ef85576ae3c9a6bd7 (Thu May 14 15:53:39 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **Invalid pull request title: No valid Jira ID provided**
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * b097f82badd3f5148a87d03842d8ee1ce3577fc6 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166230440) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] aljoscha commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
aljoscha commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-658793273


   Ahh, this discussion now reminded me of this older issue: https://issues.apache.org/jira/browse/FLINK-11654. Can you ask Jiangjie Qin there if he's still working on the issue? If not maybe you can take over.


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 07dcda24cfb9baa75b14b5ec953eabb9603e3082 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166205499) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384) 
   * b097f82badd3f5148a87d03842d8ee1ce3577fc6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * b097f82badd3f5148a87d03842d8ee1ce3577fc6 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166230440) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403) 
   * 9fa966f648aa7d97e6000938970039e83b8e73f2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 9fa966f648aa7d97e6000938970039e83b8e73f2 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166258938) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419) 
   * 45572c3aec3b7ea7664c53e2a9b17cff2912ba24 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] dawidwys commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
dawidwys commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-635556812


   @flinkbot run azure


----------------------------------------------------------------
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.

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



[GitHub] [flink] aljoscha commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
aljoscha commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-659493329


   👌 If Jiangjie doesn't respond after a while, ping me on the issue and I can assign you.


----------------------------------------------------------------
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.

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



[GitHub] [flink] aljoscha closed pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
aljoscha closed pull request #12157:
URL: https://github.com/apache/flink/pull/12157


   


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628725851


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 45572c3aec3b7ea7664c53e2a9b17cff2912ba24 (Fri Feb 19 07:26:31 UTC 2021)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * f0bdd24a7d955fb17aadc7c022ab2bd06461f869 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166171453) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359) 
   * 409246b65de6c5e76e6462ed4a55d73c7dc76c22 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] aljoscha commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
aljoscha commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-698783730


   I'm closing this as "Abandoned", since there is no more activity and the code base has moved on quite a bit. Please re-open this if you feel otherwise and work should continue.
   
   I'm changing the issue to refer to only the modern Kafka connector because Kafka 0.11 was removed by now.


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121",
       "triggerID" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4791",
       "triggerID" : "635556812",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121",
       "triggerID" : "635556812",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 45572c3aec3b7ea7664c53e2a9b17cff2912ba24 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4791) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 409246b65de6c5e76e6462ed4a55d73c7dc76c22 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166181069) Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366) 
   * 07dcda24cfb9baa75b14b5ec953eabb9603e3082 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * b097f82badd3f5148a87d03842d8ee1ce3577fc6 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166230440) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403) 
   * 9fa966f648aa7d97e6000938970039e83b8e73f2 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/166258938) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] aljoscha commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
aljoscha commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-698783730


   I'm closing this as "Abandoned", since there is no more activity and the code base has moved on quite a bit. Please re-open this if you feel otherwise and work should continue.
   
   I'm changing the issue to refer to only the modern Kafka connector because Kafka 0.11 was removed by now.


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121",
       "triggerID" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 45572c3aec3b7ea7664c53e2a9b17cff2912ba24 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166161943) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350) 
   * f0bdd24a7d955fb17aadc7c022ab2bd06461f869 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/166171453) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 4d9d19dd8a1bcaa317dc8f0550186bba13f7c103 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166056411) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] dawidwys commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
dawidwys commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-635555610


   Not necessarily locally but you can also run the CI/CD on your azure account for your flink branch. You can find instructions how to do it here: https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 9fa966f648aa7d97e6000938970039e83b8e73f2 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166258938) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] freezhan edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
freezhan edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-656483534


   In addition, our internal temporary repair method is
   
   ```java
   this.prefix = prefix.substring(0, Math.min(prefix.length(), 200)) + "-" + EncodingUtils.hex(EncodingUtils.md5(prefix));
   ```
   how about that? 
   
   Can add this sentence before calling the `TransactionalIdsGenerator ` constructor, and then add a length check inside the constructor.
   
   But I still think it makes sense to let users pass it in.
   
   If the user does not specify the prefix parameter, the default is a whole bunch of context characters(Keep it the same as before)


----------------------------------------------------------------
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.

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



[GitHub] [flink] aljoscha commented on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
aljoscha commented on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-647605986


   @freezhan Thanks a lot for analysing this bug! I think it would be better to limit the prefix instead of taking the md5 hash. The prefix potentially has some useful information that could help debugging problems with transactional IDs. I would suggest we instead cap the task name at some reasonable length before calling the `TransactionalIdsGenerator` constructor (otherwise we would also be cutting of the operator ID. And we should then also add a check inside the generator to verify that we really obey the length cap. What do you think?


----------------------------------------------------------------
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.

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



[GitHub] [flink] aljoscha closed pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
aljoscha closed pull request #12157:
URL: https://github.com/apache/flink/pull/12157


   


----------------------------------------------------------------
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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 409246b65de6c5e76e6462ed4a55d73c7dc76c22 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166181069) Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366) 
   * 07dcda24cfb9baa75b14b5ec953eabb9603e3082 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/166205499) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #12157: [FLINK-17691][Connectors/Kafka] fix the bug FlinkKafkaProducer011 transactional.id too long when use Semantic.EXACTLY_ONCE mode

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12157:
URL: https://github.com/apache/flink/pull/12157#issuecomment-628732224


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c72006072a887727d7ac814ef85576ae3c9a6bd7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166056411",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1326",
       "triggerID" : "4d9d19dd8a1bcaa317dc8f0550186bba13f7c103",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166161943",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1350",
       "triggerID" : "190d48b9cf54b6e7a53c5de43f4edbb927bf0c8e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166171453",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1359",
       "triggerID" : "f0bdd24a7d955fb17aadc7c022ab2bd06461f869",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166181069",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1366",
       "triggerID" : "409246b65de6c5e76e6462ed4a55d73c7dc76c22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1384",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166205499",
       "triggerID" : "07dcda24cfb9baa75b14b5ec953eabb9603e3082",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166230440",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1403",
       "triggerID" : "b097f82badd3f5148a87d03842d8ee1ce3577fc6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/166258938",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419",
       "triggerID" : "9fa966f648aa7d97e6000938970039e83b8e73f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121",
       "triggerID" : "45572c3aec3b7ea7664c53e2a9b17cff2912ba24",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c72006072a887727d7ac814ef85576ae3c9a6bd7 UNKNOWN
   * 9fa966f648aa7d97e6000938970039e83b8e73f2 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/166258938) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1419) 
   * 45572c3aec3b7ea7664c53e2a9b17cff2912ba24 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2121) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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