Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-21885

HiveMetastoreCatalog.InferIfNeeded too slow when caseSensitiveInference enabled

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Major
    • Resolution: Incomplete
    • 2.1.0, 2.2.0, 2.3.0
    • None
    • SQL
    • `spark.sql.hive.caseSensitiveInferenceMode` set to INFER_ONLY

    • Important

    Description

      Currently, SparkSQL infer schema is too slow, almost take 2 minutes.
      I digged into the code, and finally findout the reason:
      1. In the analysis process of LogicalPlan spark will try to infer table schema if `spark.sql.hive.caseSensitiveInferenceMode` set to INFER_ONLY, and it will list all the leaf files of the rootPaths(just tableLocation), and then call `getFileBlockLocations` to turn `FileStatus` into `LocatedFileStatus`. This `getFileBlockLocations` for so manny leaf files will take a long time, and it seems that the locations info is never used.
      2. When infer a parquet schema, if there is only one file, it will still launch a spark job to merge schema. I think it's expensive.

      Time costly stack is as follow:

      at org.apache.hadoop.ipc.Client.call(Client.java:1403)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
              at com.sun.proxy.$Proxy16.getBlockLocations(Unknown Source)
              at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(ClientNamenodeProtocolTranslatorPB.java:259)
              at sun.reflect.GeneratedMethodAccessor15.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.lang.reflect.Method.invoke(Method.java:606)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
              at com.sun.proxy.$Proxy17.getBlockLocations(Unknown Source)
              at org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1234)
              at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1224)
              at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1274)
              at org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
              at org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
              at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
              at org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:217)
              at org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:209)
              at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningAwareFileIndex$$listLeafFiles$3.apply(PartitioningAwareFileIndex.scala:427)
              at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningAwareFileIndex$$listLeafFiles$3.apply(PartitioningAwareFileIndex.scala:410)
              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.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
              at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
              at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
              at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
              at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex$.org$apache$spark$sql$execution$datasources$PartitioningAwareFileIndex$$listLeafFiles(PartitioningAwareFileIndex.scala:410)
              at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningAwareFileIndex$$bulkListLeafFiles$1.apply(PartitioningAwareFileIndex.scala:302)
              at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningAwareFileIndex$$bulkListLeafFiles$1.apply(PartitioningAwareFileIndex.scala:301)
              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.datasources.PartitioningAwareFileIndex$.org$apache$spark$sql$execution$datasources$PartitioningAwareFileIndex$$bulkListLeafFiles(PartitioningAwareFileIndex.scala:301)
              at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.listLeafFiles(PartitioningAwareFileIndex.scala:253)
              at org.apache.spark.sql.execution.datasources.InMemoryFileIndex.refresh0(InMemoryFileIndex.scala:74)
              at org.apache.spark.sql.execution.datasources.InMemoryFileIndex.<init>(InMemoryFileIndex.scala:50)
              at org.apache.spark.sql.execution.datasources.PrunedInMemoryFileIndex.<init>(CatalogFileIndex.scala:108)
              at org.apache.spark.sql.execution.datasources.CatalogFileIndex.filterPartitions(CatalogFileIndex.scala:76)
              at org.apache.spark.sql.execution.datasources.CatalogFileIndex.listFiles(CatalogFileIndex.scala:54)
              at org.apache.spark.sql.hive.HiveMetastoreCatalog.org$apache$spark$sql$hive$HiveMetastoreCatalog$$inferIfNeeded(HiveMetastoreCatalog.scala:312)
              at org.apache.spark.sql.hive.HiveMetastoreCatalog$$anonfun$5.apply(HiveMetastoreCatalog.scala:243)
              at org.apache.spark.sql.hive.HiveMetastoreCatalog$$anonfun$5.apply(HiveMetastoreCatalog.scala:229)
              at scala.Option.getOrElse(Option.scala:121)
              at org.apache.spark.sql.hive.HiveMetastoreCatalog.org$apache$spark$sql$hive$HiveMetastoreCatalog$$convertToLogicalRelation(HiveMetastoreCatalog.scala:229)
              at org.apache.spark.sql.hive.HiveMetastoreCatalog$ParquetConversions$.org$apache$spark$sql$hive$HiveMetastoreCatalog$ParquetConversions$$convertToParquetRelation(HiveMetastoreCatalog.scala:357)
              at org.apache.spark.sql.hive.HiveMetastoreCatalog$ParquetConversions$$anonfun$apply$1.applyOrElse(HiveMetastoreCatalog.scala:374)
              at org.apache.spark.sql.hive.HiveMetastoreCatalog$ParquetConversions$$anonfun$apply$1.applyOrElse(HiveMetastoreCatalog.scala:365)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:310)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:310)
              at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:309)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:307)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:307)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:307)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:307)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:307)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
      
      

      Attachments

        Activity

          People

            Unassigned Unassigned
            liupengcheng liupengcheng
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: