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

Alexander Alexandrov commented on FLINK-20043:
----------------------------------------------

Not really, even for a simple Kinesis source → Print sink dataflow such as


{code:sql}
 CREATE TABLE `mm-10139-source` (
  `event_time` TIMESTAMP(3) NOT NULL,
  `name` VARCHAR(32) NOT NULL,
  `age` BIGINT NOT NULL,
  `office` VARCHAR(255) NOT NULL,
  `role` VARCHAR(4) NOT NULL,
  `arrival_time` TIMESTAMP(3) METADATA FROM 'timestamp' VIRTUAL,
  `shard_id` VARCHAR(128) NOT NULL METADATA FROM 'shard-id' VIRTUAL,
  WATERMARK FOR `event_time` AS `event_time` - INTERVAL '5' second
) PARTITIONED BY (office, role) WITH (
  'connector' = 'kinesis',
  'stream' = 'mm-10139-source',
  'aws.region' = 'us-east-2',
  'scan.stream.initpos' = 'LATEST',
  'sink.partitioner-field-delimiter' = ';',
  'sink.producer.collection-max-count' = '100',
  'format' = 'json',
  'json.timestamp-format.standard' = 'ISO-8601'
);

CREATE TABLE `print-source` (
  `event_time` TIMESTAMP(3),
  `name` VARCHAR(32),
  `age` BIGINT,
  `office` VARCHAR(255),
  `role` VARCHAR(4)
) WITH (
  'connector' = 'print',
  'print-identifier' = 'source'
);

INSERT INTO `print-source`
SELECT
  `event_time`,
  `name`,
  `age`,
  `office`,
  `role`
FROM
  `mm-10139-source`;{code}

I get the following error

{code}
org.apache.flink.runtime.JobException: Recovery is suppressed by 
NoRestartBackoffTimeStrategy
        at 
org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:116)
        at 
org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:78)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:224)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:217)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:208)
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:534)
        at 
org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:89)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:419)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:286)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:201)
        at 
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:154)
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
        at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
        at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
        at akka.actor.ActorCell.invoke(ActorCell.scala:561)
        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
        at akka.dispatch.Mailbox.run(Mailbox.scala:225)
        at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
        at 
akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
        at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
        at 
akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
Caused by: java.lang.NoClassDefFoundError: org/apache/commons/logging/LogFactory
        at 
org.apache.flink.kinesis.shaded.com.amazonaws.ClientConfiguration.<clinit>(ClientConfiguration.java:47)
        at 
org.apache.flink.kinesis.shaded.com.amazonaws.ClientConfigurationFactory.getDefaultConfig(ClientConfigurationFactory.java:46)
        at 
org.apache.flink.kinesis.shaded.com.amazonaws.ClientConfigurationFactory.getConfig(ClientConfigurationFactory.java:36)
        at 
org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy.createKinesisClient(KinesisProxy.java:216)
        at 
org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy.<init>(KinesisProxy.java:148)
        at 
org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy.create(KinesisProxy.java:228)
        at 
org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher.<init>(KinesisDataFetcher.java:395)
        at 
org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher.<init>(KinesisDataFetcher.java:350)
        at 
org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer.createFetcher(FlinkKinesisConsumer.java:471)
        at 
org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer.run(FlinkKinesisConsumer.java:287)
        at 
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:100)
        at 
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:63)
        at 
org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:215)
Caused by: java.lang.ClassNotFoundException: 
org.apache.commons.logging.LogFactory
        at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
        at 
org.apache.flink.util.FlinkUserCodeClassLoader.loadClassWithoutExceptionHandling(FlinkUserCodeClassLoader.java:63)
        at 
org.apache.flink.util.ChildFirstClassLoader.loadClassWithoutExceptionHandling(ChildFirstClassLoader.java:63)
        at 
org.apache.flink.util.FlinkUserCodeClassLoader.loadClass(FlinkUserCodeClassLoader.java:49)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
{code}

> Add flink-sql-connector-kinesis package
> ---------------------------------------
>
>                 Key: FLINK-20043
>                 URL: https://issues.apache.org/jira/browse/FLINK-20043
>             Project: Flink
>          Issue Type: Improvement
>            Reporter: Alexander Alexandrov
>            Assignee: Alexander Alexandrov
>            Priority: Major
>         Attachments: third-party-report.html
>
>
> This is a follow up item after the recent addition of [Kinesis SQL source and 
> sink in PR #13770|https://github.com/apache/flink/pull/13770].
> Create a package that bundles a fat connector jar that can be used by SQL 
> clients. See FLINK-11026 and the related PRs for a discussion how to do that.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to