[
https://issues.apache.org/jira/browse/FLINK-5898?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16017653#comment-16017653
]
Scott Kidder commented on FLINK-5898:
-------------------------------------
I created a new build of Flink that uses KPL {{0.12.4}} and AWS SDK
{{1.11.128}}. I had a job that was unable to restore from an earlier checkpoint
made with my patched KPL {{0.12.3}} and AWS SDK {{1.11.86}}:
{noformat}
java.io.InvalidClassException: com.amazonaws.services.kinesis.model.Shard;
local class incompatible: stream classdesc serialVersionUID =
2061877776249602915, local class serialVersionUID = 5010840014163691006
at java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:616)
at
java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1829)
at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1713)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1986)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1535)
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2231)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2155)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2013)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1535)
at java.io.ObjectInputStream.readObject(ObjectInputStream.java:422)
at java.util.HashMap.readObject(HashMap.java:1402)
at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at
java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1058)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2122)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2013)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1535)
at java.io.ObjectInputStream.readObject(ObjectInputStream.java:422)
at
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:307)
at
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:166)
at
org.apache.flink.streaming.api.operators.AbstractStreamOperator.restoreStreamCheckpointed(AbstractStreamOperator.java:240)
at
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:203)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:654)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:641)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:247)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:655)
at java.lang.Thread.run(Thread.java:745)
{noformat}
So, there are incompatible changes in the Kinesis {{Shard}} class included in
the AWS SDK release referenced directly by KPL {{0.12.4}}. Just something to be
aware of when upgrading the KPL.
> 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)