Uploaded image for project: 'Apache Hudi'
  1. Apache Hudi
  2. HUDI-4218

The tableExists method should print the correct exception information when an exception occurs

    XMLWordPrintableJSON

Details

    Description

      In the existing version, when an exception occurs in this method, the following exception information will be reported regardless of the reason:

       
      Exception in thread "main" org.apache.hudi.exception.HoodieException: Failed to get Schema through jdbc.
              at org.apache.hudi.utilities.schema.JdbcbasedSchemaProvider.getSourceSchema(JdbcbasedSchemaProvider.java:81)
              at org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor.getSourceSchema(SchemaProviderWithPostProcessor.java:42)
              at org.apache.hudi.utilities.deltastreamer.DeltaSync.registerAvroSchemas(DeltaSync.java:860)
              at org.apache.hudi.utilities.deltastreamer.DeltaSync.<init>(DeltaSync.java:235)
              at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer$DeltaSyncService.<init>(HoodieDeltaStreamer.java:654)
              at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.<init>(HoodieDeltaStreamer.java:143)
              at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.<init>(HoodieDeltaStreamer.java:116)
              at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:553)
              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.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
              at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:845)
              at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:161)
              at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:184)
              at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
              at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:920)
              at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:929)
              at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
      Caused by: org.apache.hudi.exception.HoodieException: test.test_source_json table does not exists!
              at org.apache.hudi.utilities.UtilHelpers.getJDBCSchema(UtilHelpers.java:439)
              at org.apache.hudi.utilities.schema.JdbcbasedSchemaProvider.getSourceSchema(JdbcbasedSchemaProvider.java:79)
              ... 19 more
       
      For example, because of the Kerberos authentication permission problem,According to the above exception information, the root cause of the exception cannot be known. It should throw the following information to facilitate us to analyze the cause
       
      Exception in thread "main" org.apache.hudi.exception.HoodieException: Failed to get Schema through jdbc.
              at org.apache.hudi.utilities.schema.JdbcbasedSchemaProvider.getSourceSchema(JdbcbasedSchemaProvider.java:81)
              at org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor.getSourceSchema(SchemaProviderWithPostProcessor.java:42)
              at org.apache.hudi.utilities.deltastreamer.DeltaSync.registerAvroSchemas(DeltaSync.java:730)
              at org.apache.hudi.utilities.deltastreamer.DeltaSync.<init>(DeltaSync.java:220)
              at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer$DeltaSyncService.<init>(HoodieDeltaStreamer.java:606)
              at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.<init>(HoodieDeltaStreamer.java:143)
              at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.<init>(HoodieDeltaStreamer.java:107)
              at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:509)
              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.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
              at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:845)
              at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:161)
              at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:184)
              at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
              at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:920)
              at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:929)
              at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
      Caused by: org.apache.hive.service.cli.HiveSQLException: Error while compiling statement: FAILED: HiveAccessControlException Permission denied: user [spark] does not have [SELECT] privilege on [test/test_source_json/*]
              at org.apache.hive.jdbc.Utils.verifySuccess(Utils.java:255)
              at org.apache.hive.jdbc.Utils.verifySuccessWithInfo(Utils.java:241)
              at org.apache.hive.jdbc.HiveStatement.execute(HiveStatement.java:254)
              at org.apache.hive.jdbc.HiveStatement.executeQuery(HiveStatement.java:392)
              at org.apache.hive.jdbc.HivePreparedStatement.executeQuery(HivePreparedStatement.java:109)
              at org.apache.hudi.utilities.UtilHelpers.getJDBCSchema(UtilHelpers.java:381)
              at org.apache.hudi.utilities.schema.JdbcbasedSchemaProvider.getSourceSchema(JdbcbasedSchemaProvider.java:79)
              ... 19 more
      Caused by: org.apache.hive.service.cli.HiveSQLException: Error while compiling statement: FAILED: HiveAccessControlException Permission denied: user [spark] does not have [SELECT] privilege on [test/test_source_json/*]
              at org.apache.hive.service.cli.operation.Operation.toSQLException(Operation.java:335)
              at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:199)
              at org.apache.hive.service.cli.operation.SQLOperation.runInternal(SQLOperation.java:262)
              at org.apache.hive.service.cli.operation.Operation.run(Operation.java:247)
              at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementInternal(HiveSessionImpl.java:575)
              at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementAsync(HiveSessionImpl.java:561)
              at sun.reflect.GeneratedMethodAccessor78.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.lang.reflect.Method.invoke(Method.java:498)
              at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:78)
              at org.apache.hive.service.cli.session.HiveSessionProxy.access$000(HiveSessionProxy.java:36)
              at org.apache.hive.service.cli.session.HiveSessionProxy$1.run(HiveSessionProxy.java:63)
              at java.security.AccessController.doPrivileged(Native Method)
              at javax.security.auth.Subject.doAs(Subject.java:422)
              at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1730)
              at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:59)
              at com.sun.proxy.$Proxy68.executeStatementAsync(Unknown Source)
              at org.apache.hive.service.cli.CLIService.executeStatementAsync(CLIService.java:315)
              at org.apache.hive.service.cli.thrift.ThriftCLIService.ExecuteStatement(ThriftCLIService.java:566)
              at org.apache.hive.service.rpc.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1557)
              at org.apache.hive.service.rpc.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1542)
              at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
              at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
              at org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:647)
              at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      Caused by: org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException: Permission denied: user [spark] does not have [SELECT] privilege on [test/test_source_json/*]
              at org.apache.ranger.authorization.hive.authorizer.RangerHiveAuthorizer.checkPrivileges(RangerHiveAuthorizer.java:483)
              at org.apache.hadoop.hive.ql.Driver.doAuthorizationV2(Driver.java:1336)
              at org.apache.hadoop.hive.ql.Driver.doAuthorization(Driver.java:1100)
              at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:709)
              at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:1869)
              at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:1816)
              at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:1811)
              at org.apache.hadoop.hive.ql.reexec.ReExecDriver.compileAndRespond(ReExecDriver.java:126)
              at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:197)
              ... 26 more

       

      update:

      Exception in thread "main" org.apache.hudi.exception.HoodieException: Failed to get Schema through jdbc. 
          at org.apache.hudi.utilities.schema.JdbcbasedSchemaProvider.getSourceSchema(JdbcbasedSchemaProvider.java:81)
          at org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor.getSourceSchema(SchemaProviderWithPostProcessor.java:42)
          at org.apache.hudi.utilities.deltastreamer.DeltaSync.registerAvroSchemas(DeltaSync.java:865)
          at org.apache.hudi.utilities.deltastreamer.DeltaSync.<init>(DeltaSync.java:235)
          at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer$DeltaSyncService.<init>(HoodieDeltaStreamer.java:658)
          at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.<init>(HoodieDeltaStreamer.java:143)
          at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.<init>(HoodieDeltaStreamer.java:116)
          at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:557)
          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.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
          at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:845)
          at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:161)
          at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:184)
          at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
          at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:920)
          at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:929)
          at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
      Caused by: org.apache.hudi.exception.HoodieException: org.apache.hive.service.cli.HiveSQLException: Error while compiling statement: FAILED: HiveAccessControlException Permission denied: user [spark] does not have [SELECT] privilege on [test/test_source_json/*]
          at org.apache.hudi.utilities.UtilHelpers.tableExists(UtilHelpers.java:405)
          at org.apache.hudi.utilities.UtilHelpers.getJDBCSchema(UtilHelpers.java:421)
          at org.apache.hudi.utilities.schema.JdbcbasedSchemaProvider.getSourceSchema(JdbcbasedSchemaProvider.java:79)
          ... 19 more
      Caused by: org.apache.hive.service.cli.HiveSQLException: Error while compiling statement: FAILED: HiveAccessControlException Permission denied: user [spark] does not have [SELECT] privilege on [test/test_source_json/*]
          at org.apache.hive.jdbc.Utils.verifySuccess(Utils.java:255)
          at org.apache.hive.jdbc.Utils.verifySuccessWithInfo(Utils.java:241)
          at org.apache.hive.jdbc.HiveStatement.execute(HiveStatement.java:254)
          at org.apache.hive.jdbc.HiveStatement.executeQuery(HiveStatement.java:392)
          at org.apache.hive.jdbc.HivePreparedStatement.executeQuery(HivePreparedStatement.java:109)
          at org.apache.hudi.utilities.UtilHelpers.tableExists(UtilHelpers.java:403)
          ... 21 more
      Caused by: org.apache.hive.service.cli.HiveSQLException: Error while compiling statement: FAILED: HiveAccessControlException Permission denied: user [spark] does not have [SELECT] privilege on [test/test_source_json/*]
          at org.apache.hive.service.cli.operation.Operation.toSQLException(Operation.java:335)
          at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:199)
          at org.apache.hive.service.cli.operation.SQLOperation.runInternal(SQLOperation.java:262)
          at org.apache.hive.service.cli.operation.Operation.run(Operation.java:247)
          at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementInternal(HiveSessionImpl.java:575)
          at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementAsync(HiveSessionImpl.java:561)
          at sun.reflect.GeneratedMethodAccessor94.invoke(Unknown Source)
          at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          at java.lang.reflect.Method.invoke(Method.java:498)
          at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:78)
          at org.apache.hive.service.cli.session.HiveSessionProxy.access$000(HiveSessionProxy.java:36)
          at org.apache.hive.service.cli.session.HiveSessionProxy$1.run(HiveSessionProxy.java:63)
          at java.security.AccessController.doPrivileged(Native Method)
          at javax.security.auth.Subject.doAs(Subject.java:422)
          at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1730)
          at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:59)
          at com.sun.proxy.$Proxy66.executeStatementAsync(Unknown Source)
          at org.apache.hive.service.cli.CLIService.executeStatementAsync(CLIService.java:315)
          at org.apache.hive.service.cli.thrift.ThriftCLIService.ExecuteStatement(ThriftCLIService.java:566)
          at org.apache.hive.service.rpc.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1557)
          at org.apache.hive.service.rpc.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1542)
          at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
          at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
          at org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:647)
          at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286)
          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
          at java.lang.Thread.run(Thread.java:748)
      Caused by: org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException: Permission denied: user [spark] does not have [SELECT] privilege on [test/test_source_json/*]
          at org.apache.ranger.authorization.hive.authorizer.RangerHiveAuthorizer.checkPrivileges(RangerHiveAuthorizer.java:483)
          at org.apache.hadoop.hive.ql.Driver.doAuthorizationV2(Driver.java:1336)
          at org.apache.hadoop.hive.ql.Driver.doAuthorization(Driver.java:1100)
          at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:709)
          at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:1869)
          at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:1816)
          at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:1811)
          at org.apache.hadoop.hive.ql.reexec.ReExecDriver.compileAndRespond(ReExecDriver.java:126)
          at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:197)
          ... 26 more

      Attachments

        Issue Links

          Activity

            People

              dongkelun 董可伦
              dongkelun 董可伦
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: