Details
-
Improvement
-
Status: Resolved
-
Major
-
Resolution: Incomplete
-
2.1.0, 2.2.0, 2.3.0
-
None
-
`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)