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

Zheng Hu commented on FLINK-14359:
----------------------------------

When run the sql-client using the packed shaded hbase lib,  I found  hbase 
client still try to load some non-shaded hbase class such as the following 
stracktrace: 
{code}
Flink SQL> select * from MyHBaseSource;
[ERROR] Could not execute SQL statement. Reason:
org.apache.flink.runtime.rest.util.RestClientException: [Internal server 
error., <Exception on server side:
org.apache.flink.runtime.client.JobSubmissionException: Failed to submit job.
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.lambda$internalSubmitJob$2(Dispatcher.java:338)
        at 
java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:822)
        at 
java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:797)
        at 
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
        at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
        at 
akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)
        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.RuntimeException: 
org.apache.flink.runtime.client.JobExecutionException: Could not set up 
JobManager
        at 
org.apache.flink.util.function.CheckedSupplier.lambda$unchecked$0(CheckedSupplier.java:36)
        at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
        ... 6 more
Caused by: org.apache.flink.runtime.client.JobExecutionException: Could not set 
up JobManager
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl.<init>(JobManagerRunnerImpl.java:152)
        at 
org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory.createJobManagerRunner(DefaultJobManagerRunnerFactory.java:84)
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.lambda$createJobManagerRunner$5(Dispatcher.java:380)
        at 
org.apache.flink.util.function.CheckedSupplier.lambda$unchecked$0(CheckedSupplier.java:34)
        ... 7 more
Caused by: org.apache.flink.runtime.client.JobExecutionException: Cannot 
initialize task 'Source: HBaseTableSource(rowkey, a, b) -> Map -> to: Tuple2 -> 
Sink: SQL Client Stream Collect Sink': Configuring the input format (null) 
failed: Cannot create connection to HBase.
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:218)
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:106)
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.createExecutionGraph(SchedulerBase.java:222)
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.createAndRestoreExecutionGraph(SchedulerBase.java:199)
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.<init>(SchedulerBase.java:188)
        at 
org.apache.flink.runtime.scheduler.LegacyScheduler.<init>(LegacyScheduler.java:65)
        at 
org.apache.flink.runtime.scheduler.LegacySchedulerFactory.createInstance(LegacySchedulerFactory.java:70)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:278)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:266)
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:98)
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:40)
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl.<init>(JobManagerRunnerImpl.java:146)
        ... 10 more
Caused by: java.lang.Exception: Configuring the input format (null) failed: 
Cannot create connection to HBase.
        at 
org.apache.flink.runtime.jobgraph.InputOutputFormatVertex.initializeOnMaster(InputOutputFormatVertex.java:80)
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:214)
        ... 21 more
Caused by: java.lang.RuntimeException: Cannot create connection to HBase.
        at 
org.apache.flink.addons.hbase.HBaseRowInputFormat.connectToTable(HBaseRowInputFormat.java:103)
        at 
org.apache.flink.addons.hbase.HBaseRowInputFormat.configure(HBaseRowInputFormat.java:68)
        at 
org.apache.flink.runtime.jobgraph.InputOutputFormatVertex.initializeOnMaster(InputOutputFormatVertex.java:77)
        ... 22 more
Caused by: java.io.IOException: java.lang.reflect.InvocationTargetException
        at 
org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:240)
        at 
org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:218)
        at 
org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:119)
        at 
org.apache.flink.addons.hbase.HBaseRowInputFormat.connectToTable(HBaseRowInputFormat.java:96)
        ... 24 more
Caused by: java.lang.reflect.InvocationTargetException
        at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
        at 
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
        at 
org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:238)
        ... 27 more
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.ClassNotFoundException: Class 
org.apache.hadoop.hbase.client.ClusterStatusListener$MulticastListener not found
        at 
org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2299)
        at 
org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.<init>(ConnectionManager.java:714)
        ... 32 more
Caused by: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class 
org.apache.hadoop.hbase.client.ClusterStatusListener$MulticastListener not found
        at 
org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2267)
        at 
org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2291)
        ... 33 more
Caused by: java.lang.ClassNotFoundException: Class 
org.apache.hadoop.hbase.client.ClusterStatusListener$MulticastListener not found
        at 
org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2171)
        at 
org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2265)
        ... 34 more

End of exception on server side>]
{code}

I found this is because the hbase-default.xml has some configurations, says it 
will try to load the class from the config key-values, such as : 
{code}
  <property>
    <name>hbase.status.listener.class</name>
    
<value>org.apache.hadoop.hbase.client.ClusterStatusListener$MulticastListener</value>
    <description>
      Implementation of the status listener with a multicast message.
    </description>
  </property>
{code}

So we need to exclude the hbase-default.xml at least (then it will try to load 
the class from the shaded hbase lib), maybe some other unrelated files in HBase 
will also need to be excluded... let me keep trying..


> Create a module called flink-sql-connector-hbase to shade HBase
> ---------------------------------------------------------------
>
>                 Key: FLINK-14359
>                 URL: https://issues.apache.org/jira/browse/FLINK-14359
>             Project: Flink
>          Issue Type: New Feature
>          Components: Connectors / HBase
>            Reporter: Jingsong Lee
>            Assignee: Zheng Hu
>            Priority: Major
>             Fix For: 1.10.0
>
>
> We need do the same thing as kafka and elasticsearch to HBase.



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

Reply via email to