Uploaded image for project: 'CarbonData'
  1. CarbonData
  2. CARBONDATA-3802

Show metacache on table with lucene created in 2.0 fails

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Minor
    • Resolution: Unresolved
    • 2.0.0
    • None
    • data-query
    • None
    • Spark 2.3.2, 2.4.5

    Description

      Show metacache on table with lucene created in 2.0 fails

      0: jdbc:hive2://10.20.255.171:23040/default> CREATE TABLE uniqdata_lucene(CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED as carbondata;
      ---------+

      Result

      ---------+
      ---------+
      No rows selected (0.188 seconds)
      0: jdbc:hive2://10.20.255.171:23040/default> LOAD DATA INPATH 'hdfs://hacluster/chetan/2000_UniqData.csv' into table uniqdata_lucene OPTIONS('DELIMITER'=',', 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1');
      ---------+

      Result

      ---------+
      ---------+
      No rows selected (2.062 seconds)
      0: jdbc:hive2://10.20.255.171:23040/default> CREATE INDEX dm4 ON TABLE uniqdata_lucene (CUST_NAME) AS 'lucene';
      ---------+

      Result

      ---------+
      ---------+
      No rows selected (2.348 seconds)
      0: jdbc:hive2://10.20.255.171:23040/default> show metacache on table uniqdata_lucene;
      Error: java.lang.NumberFormatException: For input string: "null" (state=,code=0)

       

      Exception -

      2020-05-07 01:58:23,471 | AUDIT | [HiveServer2-Background-Pool: Thread-516] | {"time":"May 7, 2020 1:58:23 AM CST","username":"anonymous","opName":"SHOW CACHE","opId":"395854897408120","opStatus":"FAILED","opTime":"318 ms","table":"NA","extraInfo":{"Exception":"java.lang.NumberFormatException","Message":"For input string: \"null\""}} | org.apache.carbondata.processing.util.Auditor.logOperationEnd(Auditor.java:97)
      2020-05-07 01:58:23,471 | ERROR | [HiveServer2-Background-Pool: Thread-516] | Error executing query, currentState RUNNING, | org.apache.spark.internal.Logging$class.logError(Logging.scala:91)
      java.lang.NumberFormatException: For input string: "null"
      at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
      at java.lang.Integer.parseInt(Integer.java:580)
      at java.lang.Integer.parseInt(Integer.java:615)
      at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272)
      at scala.collection.immutable.StringOps.toInt(StringOps.scala:29)
      at org.apache.spark.sql.execution.command.cache.CarbonShowCacheCommand$$anonfun$org$apache$spark$sql$execution$command$cache$CarbonShowCacheCommand$$collectDriverMetaCacheInfo$1$$anonfun$applyOrElse$1.apply(CarbonShowCacheCommand.scala:446)
      at org.apache.spark.sql.execution.command.cache.CarbonShowCacheCommand$$anonfun$org$apache$spark$sql$execution$command$cache$CarbonShowCacheCommand$$collectDriverMetaCacheInfo$1$$anonfun$applyOrElse$1.apply(CarbonShowCacheCommand.scala:444)
      at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
      at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
      at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
      at scala.collection.AbstractTraversable.map(Traversable.scala:104)
      at org.apache.spark.sql.execution.command.cache.CarbonShowCacheCommand$$anonfun$org$apache$spark$sql$execution$command$cache$CarbonShowCacheCommand$$collectDriverMetaCacheInfo$1.applyOrElse(CarbonShowCacheCommand.scala:443)
      at org.apache.spark.sql.execution.command.cache.CarbonShowCacheCommand$$anonfun$org$apache$spark$sql$execution$command$cache$CarbonShowCacheCommand$$collectDriverMetaCacheInfo$1.applyOrElse(CarbonShowCacheCommand.scala:427)
      at scala.PartialFunction$$anonfun$runWith$1.apply(PartialFunction.scala:141)
      at scala.PartialFunction$$anonfun$runWith$1.apply(PartialFunction.scala:140)
      at scala.collection.Iterator$class.foreach(Iterator.scala:893)
      at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
      at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
      at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
      at scala.collection.TraversableLike$class.collect(TraversableLike.scala:271)
      at scala.collection.AbstractTraversable.collect(Traversable.scala:104)
      at org.apache.spark.sql.execution.command.cache.CarbonShowCacheCommand.org$apache$spark$sql$execution$command$cache$CarbonShowCacheCommand$$collectDriverMetaCacheInfo(CarbonShowCacheCommand.scala:427)
      at org.apache.spark.sql.execution.command.cache.CarbonShowCacheCommand.getTableCacheFromDriver(CarbonShowCacheCommand.scala:253)
      at org.apache.spark.sql.execution.command.cache.CarbonShowCacheCommand.processMetadata(CarbonShowCacheCommand.scala:105)
      at org.apache.spark.sql.execution.command.MetadataCommand$$anonfun$run$1.apply(package.scala:123)
      at org.apache.spark.sql.execution.command.MetadataCommand$$anonfun$run$1.apply(package.scala:123)
      at org.apache.spark.sql.execution.command.Auditable$class.runWithAudit(package.scala:104)
      at org.apache.spark.sql.execution.command.MetadataCommand.runWithAudit(package.scala:120)
      at org.apache.spark.sql.execution.command.MetadataCommand.run(package.scala:123)
      at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
      at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
      at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
      at org.apache.spark.sql.Dataset$$anonfun$6.apply(Dataset.scala:190)
      at org.apache.spark.sql.Dataset$$anonfun$6.apply(Dataset.scala:190)
      at org.apache.spark.sql.Dataset$$anonfun$52.apply(Dataset.scala:3259)
      at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:77)
      at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3258)
      at org.apache.spark.sql.Dataset.<init>(Dataset.scala:190)
      at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:75)
      at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:642)
      at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:694)
      at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:232)
      at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:175)
      at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:171)
      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:1698)
      at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:185)
      at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      at java.lang.Thread.run(Thread.java:745)

      Attachments

        Activity

          People

            Unassigned Unassigned
            chetdb Chetan Bhat
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated: