You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Tzu-Li (Gordon) Tai (JIRA)" <ji...@apache.org> on 2017/02/24 04:53:44 UTC

[jira] [Comment Edited] (FLINK-5898) Race-Condition with Amazon Kinesis KPL

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

Tzu-Li (Gordon) Tai edited comment on FLINK-5898 at 2/24/17 4:53 AM:
---------------------------------------------------------------------

Thanks for looking into the issue [~skidder].

This seems tricky. It isn't possible to share the {{KinesisProducer}} across the subtasks, and there's no means to coordinate multiple subtasks to synchronize this access either.

I'm not sure how we should deal with this one ...
It does however bring up the question again of whether or not we should use the low-level Java SDK instead of KPL for implementation of {{FlinkKinesisProducer}}.
[~rmetzger] what do you think?

If there is a possible way to solve this without replacing KPL and is within our reach, then I'm against considering the replacement. Right now I just don't see a possible solution other than KPL changing the binary file to be different across processes, but that's not something we can really push.


was (Author: tzulitai):
Thanks for looking into the issue [~skidder].

This seems tricky. It isn't possible to share the {{KinesisProducer}} across the subtasks, and there's no means to coordinate multiple subtasks to synchronize this access either.

I'm not sure how we should deal with this one ...
It does however bring up the question again of whether or not we should use the low-level Java SDK instead of KPL for implementation of {{FlinkKinesisProducer}}.
[~rmetzger] what do you think?

> Race-Condition with Amazon Kinesis KPL
> --------------------------------------
>
>                 Key: FLINK-5898
>                 URL: https://issues.apache.org/jira/browse/FLINK-5898
>             Project: Flink
>          Issue Type: Bug
>          Components: Kinesis Connector
>    Affects Versions: 1.2.0
>            Reporter: Scott Kidder
>
> The Flink Kinesis streaming-connector uses the Amazon Kinesis Producer Library (KPL) to send messages to Kinesis streams. The KPL relies on a native binary client to send messages to achieve better performance.
> When a Kinesis Producer is instantiated, the KPL will extract the native binary to a sub-directory of `/tmp` (or whatever the platform-specific temporary directory happens to be).
> The KPL tries to prevent multiple processes from extracting the binary at the same time by wrapping the operation in a mutex. Unfortunately, this does not prevent multiple Flink cores from trying to perform this operation at the same time. If two or more processes attempt to do this at the same time, then the native binary in /tmp will be corrupted.
> The authors of the KPL are aware of this possibility and suggest that users of the KPL .... not do that ... (sigh):
> https://github.com/awslabs/amazon-kinesis-producer/issues/55#issuecomment-251408897
> I encountered this in my production environment when bringing up a new Flink task-manager with multiple cores and restoring from an earlier savepoint, resulting in the instantiation of a KPL client on each core at roughly the same time.
> A stack-trace follows:
> {noformat}
> java.lang.RuntimeException: Could not copy native binaries to temp directory /tmp/amazon-kinesis-producer-native-binaries
> 	at com.amazonaws.services.kinesis.producer.KinesisProducer.extractBinaries(KinesisProducer.java:849)
> 	at com.amazonaws.services.kinesis.producer.KinesisProducer.<init>(KinesisProducer.java:243)
> 	at org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer.open(FlinkKinesisProducer.java:198)
> 	at org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:36)
> 	at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:112)
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.openAllOperators(StreamTask.java:376)
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:252)
> 	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:655)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.SecurityException: The contents of the binary /tmp/amazon-kinesis-producer-native-binaries/kinesis_producer_e9a87c761db92a73eb74519a4468ee71def87eb2 is not what it's expected to be.
> 	at com.amazonaws.services.kinesis.producer.KinesisProducer.extractBinaries(KinesisProducer.java:822)
> 	... 8 more
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)