Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-5013

Flink Kinesis connector doesn't work on old EMR versions

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Critical
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.1.4
    • Component/s: Kinesis Connector
    • Labels:
      None

      Description

      A user reported on the mailing list that our Kinesis connector doesn't work with EMR 4.4.0: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Kinesis-Connector-Dependency-Problems-td9790.html

      The problem seems to be that Flink is loading older libraries from the "YARN container classpath", which on EMR contains the default Amazon libraries.

      We should try to shade kinesis and its amazon dependencies into a different namespace.

        Issue Links

          Activity

          Hide
          tzulitai Tzu-Li (Gordon) Tai added a comment - - edited

          Resolved for master with http://git-wip-us.apache.org/repos/asf/flink/commit/53aae50
          Resolved for release-1.1 with http://git-wip-us.apache.org/repos/asf/flink/commit/723ce7

          Hi Craig Foster, thanks a lot for looking at this on the EMR side! I'm marking this ticket as resolved now, as the "working with old EMR version" part of the problem is solved. If you can keep us updated on the flink-dist shading issue, that would be great.

          Show
          tzulitai Tzu-Li (Gordon) Tai added a comment - - edited Resolved for master with http://git-wip-us.apache.org/repos/asf/flink/commit/53aae50 Resolved for release-1.1 with http://git-wip-us.apache.org/repos/asf/flink/commit/723ce7 Hi Craig Foster , thanks a lot for looking at this on the EMR side! I'm marking this ticket as resolved now, as the "working with old EMR version" part of the problem is solved. If you can keep us updated on the flink-dist shading issue, that would be great.
          Hide
          foscraig Craig Foster added a comment -

          Is there a fix/issue pending for correctly shading in flink-dist? What work is involved in there? The issue potentially affects downstream users such as people using BigTop and not just EMR.

          Show
          foscraig Craig Foster added a comment - Is there a fix/issue pending for correctly shading in flink-dist? What work is involved in there? The issue potentially affects downstream users such as people using BigTop and not just EMR.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/2787

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/2787
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2787

          Merging ...

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2787 Merging ...
          Hide
          rmetzger Robert Metzger added a comment -

          We only strictly enforce the maven version when building a release.
          We don't want to force regular users to use an outdated Maven version for building Flink.

          Show
          rmetzger Robert Metzger added a comment - We only strictly enforce the maven version when building a release. We don't want to force regular users to use an outdated Maven version for building Flink.
          Hide
          greghogan Greg Hogan added a comment -

          The parent POM enforces the Maven version but only for the release profile.

          Show
          greghogan Greg Hogan added a comment - The parent POM enforces the Maven version but only for the release profile.
          Hide
          foscraig Craig Foster added a comment -

          I can fix and verify on the EMR side but shouldn't the Maven enforcer plugin be used to prevent Flink from building using versions that don't properly shade?

          Show
          foscraig Craig Foster added a comment - I can fix and verify on the EMR side but shouldn't the Maven enforcer plugin be used to prevent Flink from building using versions that don't properly shade?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rmetzger commented on the issue:

          https://github.com/apache/flink/pull/2787

          I agree that we should try to get this into the RC.
          +1 to merge

          Show
          githubbot ASF GitHub Bot added a comment - Github user rmetzger commented on the issue: https://github.com/apache/flink/pull/2787 I agree that we should try to get this into the RC. +1 to merge
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2787

          @StephanEwen @rmetzger
          I would personally like to get this fix in 1.1.4 (from some recent PRs it seems like we need a new RC, so might as well see if we want to get this fix in too).

          The shading solves the problem for old EMR version users. For the other `HttpConnectionParams.setSoKeepalive` exception we probably need to wait for AWS to resolve that anyway, so it might make sense to get this in sooner?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2787 @StephanEwen @rmetzger I would personally like to get this fix in 1.1.4 (from some recent PRs it seems like we need a new RC, so might as well see if we want to get this fix in too). The shading solves the problem for old EMR version users. For the other `HttpConnectionParams.setSoKeepalive` exception we probably need to wait for AWS to resolve that anyway, so it might make sense to get this in sooner?
          Hide
          rmetzger Robert Metzger added a comment -

          Thanks a lot for looking into the issue Gordon. Great work!

          I think we should contact Amazon and see if they can resolve the issue. @Craig Foster and Steffen Hausmann can you point the EMR team to this JIRA?

          Show
          rmetzger Robert Metzger added a comment - Thanks a lot for looking into the issue Gordon. Great work! I think we should contact Amazon and see if they can resolve the issue. @ Craig Foster and Steffen Hausmann can you point the EMR team to this JIRA?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2787

          I've actually found out the reason for `HttpConnectionParams.setSoKeepalive` error, noted in the last comment in JIRA. Cross-posting it here so we don't need to jump around two places:

          It seems like that the `flink-dist` jar that comes natively with EMR 5.1.0 hasn't shaded `httpclient` / `httpcore` properly (it contains the wrong unshaded classes). On other EMR versions where we need to download the Flink binaries ourselves, the class `HttpConnectionParams` was correctly loaded from user jars (this is the expected behaviour because our distributed Flink binaries has the classes shaded in the Hadoop dependency; see [1]). However, on EMR 5.1.0, it was loading the wrong unshaded class from their own shipped `flink-dist` jar.

          Is it possible that AWS have built Flink themselves for EMR 5.1.0, and used Maven 3.3.x which lead to incorrect shading of the apache http classes?

          [1] http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Kinesis-connector-classpath-issue-when-running-Flink-1-1-SNAPSHOT-on-YARN-td7611.html

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2787 I've actually found out the reason for `HttpConnectionParams.setSoKeepalive` error, noted in the last comment in JIRA. Cross-posting it here so we don't need to jump around two places: It seems like that the `flink-dist` jar that comes natively with EMR 5.1.0 hasn't shaded `httpclient` / `httpcore` properly (it contains the wrong unshaded classes). On other EMR versions where we need to download the Flink binaries ourselves, the class `HttpConnectionParams` was correctly loaded from user jars (this is the expected behaviour because our distributed Flink binaries has the classes shaded in the Hadoop dependency; see [1] ). However, on EMR 5.1.0, it was loading the wrong unshaded class from their own shipped `flink-dist` jar. Is it possible that AWS have built Flink themselves for EMR 5.1.0, and used Maven 3.3.x which lead to incorrect shading of the apache http classes? [1] http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Kinesis-connector-classpath-issue-when-running-Flink-1-1-SNAPSHOT-on-YARN-td7611.html
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/2787

          One thought on the `HttpConnectionParams.setSoKeepalive` - we had issues with Hadoop pulling in a wrong version of that class.
          I think we shaded it in Flink's Hadoop dependency, but it might be that EMR puts is own Hadoop in the classpath, bringing the wrong class back.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/2787 One thought on the `HttpConnectionParams.setSoKeepalive` - we had issues with Hadoop pulling in a wrong version of that class. I think we shaded it in Flink's Hadoop dependency, but it might be that EMR puts is own Hadoop in the classpath, bringing the wrong class back.
          Hide
          tzulitai Tzu-Li (Gordon) Tai added a comment - - edited

          Update:
          I've found the problem with the NoSuchMethodError - HttpConnectionParams. It seems like that the flink-dist jar that comes natively with EMR 5.1.0 hasn't shaded httpclient / httpcore properly. On other EMR versions where we need to download the Flink binaries ourselves, the class HttpConnectionParams was correctly loaded from user jars (this is the expected behaviour; see [1]). However, on EMR 5.1.0, it was loading the class from the shipped flink-dist jar.

          Is it possible that AWS have built Flink themselves for EMR 5.1.0, and used Maven 3.3.x which lead to incorrect shading?

          So, to be clear, users reported two different kinds of dependency issues:
          (A) NoSuchMethodError: com.amazonaws.SDKGlobalConfiguration.isInRegionOptimizedModeEnabled
          (B) NoSuchMethodError: org.apache.http.params.HttpConnectionParams.setSoKeepalive

          (A) is verified to be fixed by shading all AWS dependencies on our Kinesis connection pom (PR fixes this).
          For (B), it only happens on EMR 5.1.0, regardless of whether we build the connector with Maven 3.0.5 or 3.3.x because the Flink dist jar wasn't properly shaded, for the suspected reason I explained above.

          [1] http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Kinesis-connector-classpath-issue-when-running-Flink-1-1-SNAPSHOT-on-YARN-td7611.html

          Show
          tzulitai Tzu-Li (Gordon) Tai added a comment - - edited Update: I've found the problem with the NoSuchMethodError - HttpConnectionParams . It seems like that the flink-dist jar that comes natively with EMR 5.1.0 hasn't shaded httpclient / httpcore properly. On other EMR versions where we need to download the Flink binaries ourselves, the class HttpConnectionParams was correctly loaded from user jars (this is the expected behaviour; see [1] ). However, on EMR 5.1.0, it was loading the class from the shipped flink-dist jar. Is it possible that AWS have built Flink themselves for EMR 5.1.0, and used Maven 3.3.x which lead to incorrect shading? So, to be clear, users reported two different kinds of dependency issues: (A) NoSuchMethodError: com.amazonaws.SDKGlobalConfiguration.isInRegionOptimizedModeEnabled (B) NoSuchMethodError: org.apache.http.params.HttpConnectionParams.setSoKeepalive (A) is verified to be fixed by shading all AWS dependencies on our Kinesis connection pom (PR fixes this). For (B), it only happens on EMR 5.1.0, regardless of whether we build the connector with Maven 3.0.5 or 3.3.x because the Flink dist jar wasn't properly shaded, for the suspected reason I explained above. [1] http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Kinesis-connector-classpath-issue-when-running-Flink-1-1-SNAPSHOT-on-YARN-td7611.html
          Hide
          fhueske Fabian Hueske added a comment -

          Two users on the mailing list reported that using Maven 3.0.x did not solve their problem: https://lists.apache.org/thread.html/ebc031a913abf7aae9244ab0d3a0d578940a1c59546c4c3b7edd2c82@%3Cuser.flink.apache.org%3E

          Show
          fhueske Fabian Hueske added a comment - Two users on the mailing list reported that using Maven 3.0.x did not solve their problem: https://lists.apache.org/thread.html/ebc031a913abf7aae9244ab0d3a0d578940a1c59546c4c3b7edd2c82@%3Cuser.flink.apache.org%3E
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user tzulitai opened a pull request:

          https://github.com/apache/flink/pull/2787

          FLINK-5013 [kinesis] Shade AWS dependencies to work with older EMR versions

          This PR adds shading to fix the reported dependency issues on older EMR versions.
          Randomly tested for 4.x EMR on versions 4.3.0, 4.4.0, 4.7.2, and 4.8.0, as well as 5.0.3. Works without issues.

          However, for the native Flink support in EMR 5.1.0, the `NoSuchMethodError` for `HttpConnectionParams.setSoKeepalive` still remains, even with the shading. I'm suspecting that its a different problem.

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/tzulitai/flink FLINK-5013

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/2787.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #2787


          commit 1c56bda9690eb145006b19ba6b4f7649704f7fd4
          Author: Tzu-Li (Gordon) Tai <tzulitai@apache.org>
          Date: 2016-11-11T09:01:11Z

          FLINK-5013 [kinesis] Shade AWS dependencies to work with older EMR versions


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user tzulitai opened a pull request: https://github.com/apache/flink/pull/2787 FLINK-5013 [kinesis] Shade AWS dependencies to work with older EMR versions This PR adds shading to fix the reported dependency issues on older EMR versions. Randomly tested for 4.x EMR on versions 4.3.0, 4.4.0, 4.7.2, and 4.8.0, as well as 5.0.3. Works without issues. However, for the native Flink support in EMR 5.1.0, the `NoSuchMethodError` for `HttpConnectionParams.setSoKeepalive` still remains, even with the shading. I'm suspecting that its a different problem. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tzulitai/flink FLINK-5013 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2787.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #2787 commit 1c56bda9690eb145006b19ba6b4f7649704f7fd4 Author: Tzu-Li (Gordon) Tai <tzulitai@apache.org> Date: 2016-11-11T09:01:11Z FLINK-5013 [kinesis] Shade AWS dependencies to work with older EMR versions
          Hide
          tzulitai Tzu-Li (Gordon) Tai added a comment - - edited

          Results of my tests:

          Shading com.amazonaws.* to a different namespace solves the reported dependency issue for 4.x (randomly tested on 4.3.0, 4.4.0, 4.7.2, 4.8.0). It also works without issue for 5.0.3.

          However, for the native Flink support in EMR 5.1.0, the NoSuchMethodError for HttpConnectionParams.setSoKeepalive still remains, even with the shading. It seems like somehow the connector is catching HttpClient 4.2.6 (Flink probably uses this version of HttpClient somewhere else), while the AWS Kinesis libraries we are using uses 4.3.6. I've tried packaging the user jar with newer HttpClient versions, but still doesn't work. Neither does packaging it with the Flink Kinesis Connector.

          The NoSuchMethodError for HttpConnectionParams.setSoKeepalive should be a different problem. Any clues what might be going on? The above was tested with Maven 3.0.5.

          Show
          tzulitai Tzu-Li (Gordon) Tai added a comment - - edited Results of my tests: Shading com.amazonaws.* to a different namespace solves the reported dependency issue for 4.x (randomly tested on 4.3.0, 4.4.0, 4.7.2, 4.8.0). It also works without issue for 5.0.3. However, for the native Flink support in EMR 5.1.0, the NoSuchMethodError for HttpConnectionParams.setSoKeepalive still remains, even with the shading. It seems like somehow the connector is catching HttpClient 4.2.6 (Flink probably uses this version of HttpClient somewhere else), while the AWS Kinesis libraries we are using uses 4.3.6. I've tried packaging the user jar with newer HttpClient versions, but still doesn't work. Neither does packaging it with the Flink Kinesis Connector. The NoSuchMethodError for HttpConnectionParams.setSoKeepalive should be a different problem. Any clues what might be going on? The above was tested with Maven 3.0.5.
          Hide
          tzulitai Tzu-Li (Gordon) Tai added a comment -

          Adding the testing for different external service versions is a good idea. I recall there was a check list when testing RCs for 1.1.0 (https://docs.google.com/document/d/1cDZGtnGJKLU1fLw8AE_FzkoDLOR8amYT2oc3mD0_lw4/edit#heading=h.2v6zy51pgj33). Perhaps we could these version checks to the list also in the future?

          Show
          tzulitai Tzu-Li (Gordon) Tai added a comment - Adding the testing for different external service versions is a good idea. I recall there was a check list when testing RCs for 1.1.0 ( https://docs.google.com/document/d/1cDZGtnGJKLU1fLw8AE_FzkoDLOR8amYT2oc3mD0_lw4/edit#heading=h.2v6zy51pgj33 ). Perhaps we could these version checks to the list also in the future?
          Hide
          tzulitai Tzu-Li (Gordon) Tai added a comment -

          This seems a bit odd to me, because I also encountered the NoSuchMethodError for HttpConnectionParams.setSoKeepalive error before, which I fixed by using Maven 3.0.x to rebuild afterwards. I was using the connector on plain EC2, not on EMR YARN. I'll take the issue and investigate further.

          Show
          tzulitai Tzu-Li (Gordon) Tai added a comment - This seems a bit odd to me, because I also encountered the NoSuchMethodError for HttpConnectionParams.setSoKeepalive error before, which I fixed by using Maven 3.0.x to rebuild afterwards. I was using the connector on plain EC2, not on EMR YARN. I'll take the issue and investigate further.
          Hide
          fhueske Fabian Hueske added a comment -

          This might also be cause by using the wrong Maven version (3.3.x instead of 3.0.3).
          I've seen these NoSuchMethodError}}s for the {{HttpConnectionParams before when using a custom Flink build on EMR (although not using Kinesis but reading files from S3).

          Show
          fhueske Fabian Hueske added a comment - This might also be cause by using the wrong Maven version (3.3.x instead of 3.0.3). I've seen these NoSuchMethodError}}s for the {{HttpConnectionParams before when using a custom Flink build on EMR (although not using Kinesis but reading files from S3).
          Hide
          till.rohrmann Till Rohrmann added a comment -

          Another user reported that on EMR 5.1.0 he gets the following error when using the FlinkKinesisConsumer:

          java.lang.NoSuchMethodError: org.apache.http.params.HttpConnectionParams.setSoKeepalive(Lorg/apache/http/params/HttpParams;Z)V
                          at com.amazonaws.http.HttpClientFactory.createHttpClient(HttpClientFactory.java:96)
                          at com.amazonaws.http.AmazonHttpClient.<init>(AmazonHttpClient.java:187)
                          at com.amazonaws.AmazonWebServiceClient.<init>(AmazonWebServiceClient.java:136)
                          at com.amazonaws.services.kinesis.AmazonKinesisClient.<init>(AmazonKinesisClient.java:221)
                          at com.amazonaws.services.kinesis.AmazonKinesisClient.<init>(AmazonKinesisClient.java:197)
                          at org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.createKinesisClient(AWSUtil.java:56)
                          at org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy.<init>(KinesisProxy.java:118)
                          at org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy.create(KinesisProxy.java:176)
                          at org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher.<init>(KinesisDataFetcher.java:188)
                          at org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer.run(FlinkKinesisConsumer.java:198)
                          at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:80)
                          at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:53)
                          at org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:56)
                          at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:266)
                          at org.apache.flink.runtime.taskmanager.Task.run(Task.java:585)
                          at java.lang.Thread.run(Thread.java:745)
          

          This seems to be also a dependency issue.

          In general it would be great if we tested connectors for these kind of platforms for different versions.

          Show
          till.rohrmann Till Rohrmann added a comment - Another user reported that on EMR 5.1.0 he gets the following error when using the FlinkKinesisConsumer : java.lang.NoSuchMethodError: org.apache.http.params.HttpConnectionParams.setSoKeepalive(Lorg/apache/http/params/HttpParams;Z)V at com.amazonaws.http.HttpClientFactory.createHttpClient(HttpClientFactory.java:96) at com.amazonaws.http.AmazonHttpClient.<init>(AmazonHttpClient.java:187) at com.amazonaws.AmazonWebServiceClient.<init>(AmazonWebServiceClient.java:136) at com.amazonaws.services.kinesis.AmazonKinesisClient.<init>(AmazonKinesisClient.java:221) at com.amazonaws.services.kinesis.AmazonKinesisClient.<init>(AmazonKinesisClient.java:197) at org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.createKinesisClient(AWSUtil.java:56) at org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy.<init>(KinesisProxy.java:118) at org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy.create(KinesisProxy.java:176) at org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher.<init>(KinesisDataFetcher.java:188) at org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer.run(FlinkKinesisConsumer.java:198) at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:80) at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:53) at org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:56) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:266) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:585) at java.lang. Thread .run( Thread .java:745) This seems to be also a dependency issue. In general it would be great if we tested connectors for these kind of platforms for different versions.
          Hide
          StephanEwen Stephan Ewen added a comment -

          +1

          Show
          StephanEwen Stephan Ewen added a comment - +1

            People

            • Assignee:
              tzulitai Tzu-Li (Gordon) Tai
              Reporter:
              rmetzger Robert Metzger
            • Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development