Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-25474][SQL] Support spark.sql.statistics.fallBackToHdfs in data source tables #22502

Closed
wants to merge 9 commits into from

Conversation

shahidki31
Copy link
Contributor

@shahidki31 shahidki31 commented Sep 20, 2018

What changes were proposed in this pull request?

In case of CatalogFileIndex datasource table, sizeInBytes is always coming as default size in bytes, which is 8.0EB (Even when the user give fallBackToHdfsForStatsEnabled=true) . So, the datasource table which has CatalogFileIndex, always prefer SortMergeJoin, instead of BroadcastJoin, even though the size is below broadcast join threshold.
In this PR, In case of CatalogFileIndex table, if we enable "fallBackToHdfsForStatsEnabled=true", then the computeStatistics get the sizeInBytes from the hdfs and we get the actual size of the table. Hence, during join operation, when the table size is below broadcast threshold, it will prefer broadCastHashJoin instead of SortMergeJoin.

How was this patch tested?

Added UT

@shahidki31 shahidki31 changed the title [SPARK-25474][SQL]Size in bytes of the query is coming in EB in case of parquet datasource [SPARK-25474][SQL]Datasource table using SortMergeJoin instead of BroadCastJoin, eventhough the size of the table is under broadcast threshold Sep 21, 2018
@shahidki31 shahidki31 changed the title [SPARK-25474][SQL]Datasource table using SortMergeJoin instead of BroadCastJoin, eventhough the size of the table is under broadcast threshold [SPARK-25474][SQL]When the "fallBackToHdfsForStatsEnabled=true", Datasource table using SortMergeJoin instead of BroadCastJoin, eventhough the size of the table is under broadcast threshold. Sep 21, 2018
@shahidki31 shahidki31 changed the title [SPARK-25474][SQL]When the "fallBackToHdfsForStatsEnabled=true", Datasource table using SortMergeJoin instead of BroadCastJoin, eventhough the size of the table is under broadcast threshold. [SPARK-25474][SQL]When the "fallBackToHdfsForStatsEnabled=true", Size in bytes is coming as default size in bytes ( 8.0 EB) in case of parquet datasource table. Sep 21, 2018
@shahidki31 shahidki31 changed the title [SPARK-25474][SQL]When the "fallBackToHdfsForStatsEnabled=true", Size in bytes is coming as default size in bytes ( 8.0 EB) in case of parquet datasource table. [SPARK-25474][SQL]When the "fallBackToHdfsForStatsEnabled = true", Size in bytes is coming as default size in bytes ( 8.0 EB) in case of parquet datasource table. Sep 21, 2018
@shahidki31
Copy link
Contributor Author

shahidki31 commented Sep 21, 2018

cc @cloud-fan @gatorsmile

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@shahidki31 . The code doesn't look specific to Parquet data source. If then, please remove in case of parquet datasource table from the title.

@shahidki31 shahidki31 changed the title [SPARK-25474][SQL]When the "fallBackToHdfsForStatsEnabled = true", Size in bytes is coming as default size in bytes ( 8.0 EB) in case of parquet datasource table. [SPARK-25474][SQL]When the "fallBackToHdfsForStatsEnabled = true", Size in bytes is coming as default size in bytes ( 8.0 EB) Sep 25, 2018
@shahidki31
Copy link
Contributor Author

shahidki31 commented Sep 25, 2018

@dongjoon-hyun . Thanks for the comment. I have modified the title. Kindly review the PR.

@shahidki31
Copy link
Contributor Author

Hi @cloud-fan , could you please review the code.

@shahidki31 shahidki31 changed the title [SPARK-25474][SQL]When the "fallBackToHdfsForStatsEnabled = true", Size in bytes is coming as default size in bytes ( 8.0 EB) [SPARK-25474][SQL]When the "fallBackToHdfsForStats= true", Size in bytes is coming as default size in bytes ( 8.0 EB) Oct 16, 2018
@cloud-fan
Copy link
Contributor

@shahidki31 thanks for fixing it!

Do you know where we read fallBackToHdfsForStats currently and see if we can have a unified place to do it?

@shahidki31
Copy link
Contributor Author

@cloud-fan Thanks. I will check and update the PR.

@SparkQA
Copy link

SparkQA commented Jan 15, 2019

Test build #101276 has finished for PR 22502 at commit 79d0794.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

}

private def sizeInBytesFallBackToHdfs: Long = {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Rather than repeat the 'compression' part here, you could inline this method, return from the try block, and ignore the exception, falling through to a default return value with the 'compression' logic

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have updated the PR, based on the above comments

…tes is coming as default size in bytes ( 8.0 EB)
…tes is coming as default size in bytes ( 8.0 EB)
@SparkQA
Copy link

SparkQA commented Jun 26, 2019

Test build #106905 has finished for PR 22502 at commit 4bef729.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

val compressionFactor = sqlContext.conf.fileCompressionFactor
(location.sizeInBytes * compressionFactor).toLong
val defaultSize = (location.sizeInBytes * compressionFactor).toLong

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

location match {
  case cfi: CatalogFileIndex if sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled =>
    DDLUtils...
  case _ => defaultSize
}

maybe?
Does it make sense to push the check for sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled into the method?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks. Updated.

Does it make sense to push the check for sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled into the method?

The method sizeInBytesFallBackToHdfs is supposed for getting sizeInBytes from hdfs, if user enable the fallback configuration. I am not sure about moving configuration check to the method sizeInBytesFallBackToHdfs.

location match {
case _: CatalogFileIndex if sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled =>
DDLUtils.sizeInBytesFallBackToHdfs(sparkSession,
new Path(location.asInstanceOf[CatalogFileIndex].table.location), defaultSize)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if you name the case match variable, you already have it as a cast here. But yeah this is cleaner.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. Updated.

@SparkQA
Copy link

SparkQA commented Jun 26, 2019

Test build #106935 has finished for PR 22502 at commit c758d42.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jun 26, 2019

Test build #106940 has finished for PR 22502 at commit ca25a17.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@srowen srowen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks reasonable to me. @dongjoon-hyun ?

@wangyum
Copy link
Member

wangyum commented Jun 27, 2019

I think the correct approach should be to add a new rule(#24715) if the issue occurs at the table level.
Actually, I have a long-term plan:

  1. Data source tables support fallback to HDFS for size estimation [SPARK-25474][SQL][DOCS] Update the docs for spark.sql.statistics.fallBackToHdfs #24715
  2. Remove duplicate logic of calculate table size [SPARK-27843][SQL] Remove duplicate logic of calculate table size #24712
  3. Persistent the table statistics to metadata after fall back to hdfs [SPARK-27655][SQL] Persistent the table statistics to metadata after fall back to hdfs #24551
  4. Refactor DetermineTableStats to invalidate cache when some configuration changed [SPARK-25740][SQL] Refactor DetermineTableStats to invalidate cache when some configuration changed #22743

For example, after #24715:

[root@spark-3267648 spark]# bin/spark-shell --conf spark.sql.statistics.fallBackToHdfs=true
Spark context Web UI available at http://spark-3267648.lvs02.dev.ebayc3.com:4040
Spark context available as 'sc' (master = local[*], app id = local-1561652081851).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_211)
Type in expressions to have them evaluated.
Type :help for more information.

scala> spark.sql("create table table1 (id int, name string) using parquet partitioned by (name)")
res0: org.apache.spark.sql.DataFrame = []

scala> spark.sql("insert into table1 values (1, 'a')")
res1: org.apache.spark.sql.DataFrame = []

scala> spark.sql("explain cost select * from table1").show(false)
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|plan                                                                                                                                                                                                                                                                                                                                                                                        |
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|== Optimized Logical Plan ==
Relation[id#2,name#3] parquet, Statistics(sizeInBytes=421.0 B)

== Physical Plan ==
*(1) FileScan parquet default.table1[id#2,name#3] Batched: true, DataFilters: [], Format: Parquet, Location: CatalogFileIndex[file:/root/opensource/spark/spark-warehouse/table1], PartitionCount: 1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>

|
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+

@shahidki31
Copy link
Contributor Author

@wangyum The issue seems happening only with the catalogFileIndex datasource tables. For InMemoryFileIndex case, sizeInBytes is already estimating from the hdfs. That is why in the PR I have put the condition for only catalogFileIndex.

override def sizeInBytes: Long = allFiles().map(_.getLen).sum

@dongjoon-hyun
Copy link
Member

Sorry for missing your ping here, @srowen . I'll take a look this one and @wangyum 's PRs together tomorrow.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Very sorry for the delay, @srowen and @shahidki31 .
I left a few comments. I'll review again after rebasing and updating.

@shahidki31
Copy link
Contributor Author

Thanks @srowen @dongjoon-hyun @maropu for the review comments. I have updated the code.

@SparkQA
Copy link

SparkQA commented Jul 28, 2019

Test build #108271 has finished for PR 22502 at commit 878c6ac.

  • This patch passes all tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 28, 2019

Test build #108272 has finished for PR 22502 at commit 4e51a4a.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 28, 2019

Test build #108277 has finished for PR 22502 at commit a27e72b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

sql("CREATE TABLE t1 (id INT, name STRING) USING PARQUET PARTITIONED BY (name)")
sql("INSERT INTO t1 VALUES (1, 'a')")
// Analyze command updates the statistics of table `t1`
sql("analyze table t1 compute statistics")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit.

- // Analyze command updates the statistics of table `t1`
- sql("analyze table t1 compute statistics")
+ sql("ANALYZE TABLE t1 COMPUTE STATISTICS")

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@shahidki31 . I'll fix this during merging~

}
}
}
assert(sizeInBytesEnabledFallBack === sizeInBytesDisabledFallBack)
Copy link
Member

@dongjoon-hyun dongjoon-hyun Jul 28, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ur, if the fallback logic returns the same value with ANALYZE TABLE t1 COMPUTE STATISTICS, this assertion doesn't prove anything. Never mind.

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-25474][SQL] When the "fallBackToHdfsForStats= true", Size in bytes is coming as default size in bytes ( 8.0 EB) [SPARK-25474][SQL] Support spark.sql.statistics.fallBackToHdfs in data source tables Jul 28, 2019
@@ -1484,4 +1484,44 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto
}
}
}

test("SPARK-25474: test sizeInBytes for CatalogFileIndex dataSourceTable") {
withSQLConf("spark.sql.statistics.fallBackToHdfs" -> "true") {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit. SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS.key.


var sizeInBytesDisabledFallBack, sizeInBytesEnabledFallBack = 0L
Seq(true, false).foreach { fallBackToHdfs =>
withSQLConf("spark.sql.statistics.fallBackToHdfs" -> fallBackToHdfs.toString) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit. SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS.key.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM. Thank you, @shahidki31 , @maropu , @srowen , and @wangyum .
Merged to master.

@dongjoon-hyun
Copy link
Member

Could you make two backporting PRs to branch-2.4 and branch-2.3, please? @shahidki31 .

@dongjoon-hyun
Copy link
Member

cc @gatorsmile and @cloud-fan

@shahidki31
Copy link
Contributor Author

Thank you @dongjoon-hyun for merging. Sure, I will create PRs for backporting.

@shahidki31 shahidki31 deleted the SPARK-25474 branch July 29, 2019 06:48
@dongjoon-hyun
Copy link
Member

Thanks!

(location.sizeInBytes * compressionFactor).toLong
val defaultSize = (location.sizeInBytes * compressionFactor).toLong
location match {
case cfi: CatalogFileIndex if sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should only fallback to HDFS if the table stats are not available from table metadata. How can we know the table stats are not available here?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I checked the code in DataSource.resolveRelation, the CatalogFileIndex is created as

val index = new CatalogFileIndex(
  sparkSession,
  catalogTable.get,
  catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize))

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice catch! Then, shall we compare the location.sizeInBytes with spark.sql.defaultSizeInBytes in order to check that?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SGTM

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think (location.sizeInBytes * compressionFactor).toLong is enough.

And (location.sizeInBytes * compressionFactor).toLong is faster than CommandUtils.getSizeInBytesFallBackToHdfs(sparkSession, new Path(cfi.table.location), defaultSize).

That why I only recalculate table statistics if we go this code path:

val defaultTableSize = sparkSession.sessionState.conf.defaultSizeInBytes
val index = new CatalogFileIndex(
sparkSession,
catalogTable.get,
catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize))

https://github.com/apache/spark/pull/24715/files#diff-d99813bd5bbc18277e4090475e4944cfR643-R646

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not sure, is there any issue in this PR. As per this code, if the table doesn't have any statistics, then only will come to the sizeInBytes method. May be we can add the extra check mentioned above.

override def computeStats(): Statistics = {
catalogTable
.flatMap(_.stats.map(_.toPlanStats(output, conf.cboEnabled)))
.getOrElse(Statistics(sizeInBytes = relation.sizeInBytes))
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IIUC the issue in this PR is, we always fallback to HDFS stats even if table stats are available.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan I have created a followup PR to add the extra condition. Thanks

Copy link
Contributor Author

@shahidki31 shahidki31 Aug 15, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. I have prepared some tests to illustrate this issue. These tests can be passed before this commit:

@wangyum First and 3rd will pass after the PR. 2nd test is a bug which fixed in the commit.

Btw, 1st and 3rd tests are not CatalogFileIndex, which any way won't come to this flow

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Aug 18, 2019

Hi, guys. As @cloud-fan mentioned, since there is a regression case, I'll revert this from branch-2.4 and branch-2.3 for 2.4.4/2.3.4 release to prevent VOTE failures. In the master branch, we need to use those test case in #24715 .

cc @kiszk since he is a release manager for 2.3.4.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
7 participants