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

A null pointer exception is thrown when data is written to the HBase table using the specified column name

    XMLWordPrintableJSON

Details

    Description

      A null pointer exception is thrown when data is written to the HBase table using the specified column name.

      The SQL examples are as follows:

      CREATE TABLE testTable4(
        rowkey INT,  
        family1 ROW<col1 INT>,
        family2 ROW<col1 VARCHAR, col2 BIGINT>,
        family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 VARCHAR>
      ) WITH (
        'connector' = 'hbase-2.2',
        'table-name' = 'testTable4',
        'zookeeper.quorum' = 'localhost:2181',
        'zookeeper.znode.parent' = '/hbase'
      )
      
      -- Specifies the column name to insert
      INSERT INTO testTable4 (rowkey, family3)  SELECT 1, ROW(1.0, false, 'c')
      

      The exception information is as follows:

      org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy
              at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:139)
              at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:83)
              at org.apache.flink.runtime.scheduler.DefaultScheduler.recordTaskFailure(DefaultScheduler.java:256)
              at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:247)
              at org.apache.flink.runtime.scheduler.DefaultScheduler.onTaskFailed(DefaultScheduler.java:240)
              at org.apache.flink.runtime.scheduler.SchedulerBase.onTaskExecutionStateUpdate(SchedulerBase.java:739)
              at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:716)
              at org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:80)
              at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:479)
              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.lambda$handleRpcInvocation$1(AkkaRpcActor.java:309)
              at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83)
              at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:307)
              at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:222)
              at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:84)
              at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:168)
              at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24)
              at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20)
              at scala.PartialFunction.applyOrElse(PartialFunction.scala:127)
              at scala.PartialFunction.applyOrElse$(PartialFunction.scala:126)
              at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20)
              at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:175)
              at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
              at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
              at akka.actor.Actor.aroundReceive(Actor.scala:537)
              at akka.actor.Actor.aroundReceive$(Actor.scala:535)
              at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220)
              at akka.actor.ActorCell.receiveMessage(ActorCell.scala:579)
              at akka.actor.ActorCell.invoke(ActorCell.scala:547)
              at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270)
              at akka.dispatch.Mailbox.run(Mailbox.scala:231)
              at akka.dispatch.Mailbox.exec(Mailbox.scala:243)
              ... 4 more
      Caused by: java.lang.NullPointerException
              at org.apache.flink.connector.hbase.util.HBaseSerde.lambda$createNullableFieldEncoder$a1c61e38$1(HBaseSerde.java:332)
              at org.apache.flink.connector.hbase.util.HBaseSerde.createPutMutation(HBaseSerde.java:147)
              at org.apache.flink.connector.hbase.sink.RowDataToMutationConverter.convertToMutation(RowDataToMutationConverter.java:53)
              at org.apache.flink.connector.hbase.sink.RowDataToMutationConverter.convertToMutation(RowDataToMutationConverter.java:32)
              at org.apache.flink.connector.hbase.sink.HBaseSinkFunction.invoke(HBaseSinkFunction.java:194)
              at org.apache.flink.table.runtime.operators.sink.SinkOperator.processElement(SinkOperator.java:65)
              at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:82)
              at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:57)
              at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29)
              at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:56)
              at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:29)
              at StreamExecCalc$4.processElement(Unknown Source)
              at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:82)
              at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:57)
              at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29)
              at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:56)
              at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:29)
              at org.apache.flink.streaming.api.operators.StreamSourceContexts$ManualWatermarkContext.processAndCollect(StreamSourceContexts.java:418)
              at org.apache.flink.streaming.api.operators.StreamSourceContexts$WatermarkContext.collect(StreamSourceContexts.java:513)
              at org.apache.flink.streaming.api.operators.StreamSourceContexts$SwitchingOnClose.collect(StreamSourceContexts.java:103)
              at org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction.run(InputFormatSourceFunction.java:92)
              at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110)
              at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:67)
              at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:333)
      

      The following pull request is my modification to this problem.

       

       

      Attachments

        Activity

          People

            wjc920 Jichao Wang
            wjc920 Jichao Wang
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: