[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-12 Thread Ran Haim (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16248837#comment-16248837
 ] 

Ran Haim commented on SPARK-22481:
--

Hi,
I create this simple test that will show how slow refresh table can be.
I create a parquet table, with 10K files just to make it obvious how slow it 
can get - but a table with a lot of partitions will have the same effect.


{code:java}
import java.util.UUID

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.SparkConf
import org.apache.spark.sql.SparkSession
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import org.scalatest.{BeforeAndAfter, FunSuite}

@RunWith(classOf[JUnitRunner])
class TestRefreshTable extends FunSuite with BeforeAndAfter{
  {
if (System.getProperty("os.name").toLowerCase.startsWith("windows")) {
  val hadoopDirURI = ClassLoader.getSystemResource("hadoop").toURI.getPath
  System.setProperty("hadoop.home.dir",hadoopDirURI )
}
  }

  var fs: FileSystem = _

  before{
fs = FileSystem.get(new Configuration())
fs.setWriteChecksum(false)
  }

  private val path = "/tmp/test"


  test("big table refresh test"){
val config = new SparkConf().
  setMaster("local[*]").
  setAppName("test").
  set("spark.ui.enabled", "false")

val sparkSession = SparkSession
  .builder()
  .config(config)
  .enableHiveSupport()
  .getOrCreate()
try {

  for (i <- 1 to 1) {
val id = UUID.randomUUID().toString
fs.create(new Path(path, id))
  }

  sparkSession.sql("drop TABLE if exists test")
  sparkSession.sql(
s"""
  CREATE EXTERNAL TABLE test(
  `rowId` bigint)
 ROW FORMAT SERDE
'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
STORED AS INPUTFORMAT
  'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
OUTPUTFORMAT
  'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
LOCATION
  '$path'
""")


  val start = System.currentTimeMillis()
  sparkSession.catalog.refreshTable("test")
  val end = System.currentTimeMillis()

  println(s"refresh took ${end - start}")
}finally {
  sparkSession.close()
}
  }
}
{code}


> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   

[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-10 Thread Ran Haim (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16247913#comment-16247913
 ] 

Ran Haim commented on SPARK-22481:
--

No, it is not.

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-10 Thread Kazuaki Ishizaki (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16247873#comment-16247873
 ] 

Kazuaki Ishizaki commented on SPARK-22481:
--

Thanks.
Is that dataframe cached or not?

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-10 Thread Ran Haim (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16247858#comment-16247858
 ] 

Ran Haim commented on SPARK-22481:
--

Ok, I'll create a simple app to reproduce this, later next week.

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-10 Thread Ran Haim (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16247852#comment-16247852
 ] 

Ran Haim commented on SPARK-22481:
--

I will double check on Sunday, but sparksession.table in the end does 
sparkSession.sessionState.executePlan(logicalPlan), and because my tables are 
parquet tables, if I am not mistaken, it will go and fetch the list of files 
from the filesystem.

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-10 Thread Kazuaki Ishizaki (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16247845#comment-16247845
 ] 

Kazuaki Ishizaki commented on SPARK-22481:
--

Sorry, I overlooked "where it used to take 2 seconds".
If you can add a program to reproduce this, it would be easy for us to analyze. 
If not, each of us may look at different problems.

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-10 Thread Ran Haim (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16247838#comment-16247838
 ] 

Ran Haim commented on SPARK-22481:
--

I can check it again on Sunday.
I don't know why it is suprising, as it has to go and fetch a list of files 
from the filesystem.

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-10 Thread Wenchen Fan (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16247836#comment-16247836
 ] 

Wenchen Fan commented on SPARK-22481:
-

Can you double check that a simple `SparkSession.table` can take 2 seconds? 
That's really surprising.

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-10 Thread Ran Haim (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16247828#comment-16247828
 ] 

Ran Haim commented on SPARK-22481:
--

It is as I wrote above.
It takes 1 minute in 2.1.1 and 2 seconds in 2.1.0.
It is unnecessary to create the dataset, unless you know it is actually cached 
- this is why it is so slow now.

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-10 Thread Kazuaki Ishizaki (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16247826#comment-16247826
 ] 

Kazuaki Ishizaki commented on SPARK-22481:
--

How long does it take in the old code?
Can you put benchmark program to compare performance before and after 2.1.1?

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-09 Thread Ran Haim (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16246176#comment-16246176
 ] 

Ran Haim commented on SPARK-22481:
--

It takes about 2 seconds to create the dataset...i need to refresh 30 tables, 
and that takes a minute now - where it used to take 2 seconds.
In the old code, it does the same because iscashed actually uses the plan and 
not the dataset.

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-09 Thread Wenchen Fan (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16246070#comment-16246070
 ] 

Wenchen Fan commented on SPARK-22481:
-

`SparkSession.table` is pretty cheap, I think the slowness is due to we now 
uncache all plans that refer to the given plan. This is for correctness, so I 
don't think it's a regression.

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-22481) CatalogImpl.refreshTable is slow

2017-11-09 Thread Kazuaki Ishizaki (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-22481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16246041#comment-16246041
 ] 

Kazuaki Ishizaki commented on SPARK-22481:
--

This change was introduce by [this 
PR|https://github.com/apache/spark/pull/17097/files#diff-463cb1b0f60d87ada075a820f18e1104R443].
 It seems to be related to the first refactoring in the description.

[~cloud_fan] Is there any reason to always call {{sparkSession.table()}}?

> CatalogImpl.refreshTable is slow
> 
>
> Key: SPARK-22481
> URL: https://issues.apache.org/jira/browse/SPARK-22481
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.1.1, 2.1.2, 2.2.0
>Reporter: Ran Haim
>Priority: Critical
>
> CatalogImpl.refreshTable was updated in 2.1.1 and since than it has become 
> really slow.
> The cause of the issue is that it is now *always* creates a dataset, and this 
> is redundant most of the time, we only need the dataset if the table is 
> cached.
> before 2.1.1:
>   override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> val logicalPlan = 
> sparkSession.sessionState.catalog.lookupRelation(tableIdent)
> // Use lookupCachedData directly since RefreshTable also takes 
> databaseName.
> val isCached = 
> sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
> if (isCached) {
>   // Create a data frame to represent the table.
>   // TODO: Use uncacheTable once it supports database name.
>  {color:red} val df = Dataset.ofRows(sparkSession, logicalPlan){color}
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(df, 
> Some(tableIdent.table))
> }
>   }
> after 2.1.1:
>override def refreshTable(tableName: String): Unit = {
> val tableIdent = 
> sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
> // Temp tables: refresh (or invalidate) any metadata/data cached in the 
> plan recursively.
> // Non-temp tables: refresh the metadata cache.
> sessionCatalog.refreshTable(tableIdent)
> // If this table is cached as an InMemoryRelation, drop the original
> // cached version and make the new version cached lazily.
> {color:red}   val table = sparkSession.table(tableIdent){color}
> if (isCached(table)) {
>   // Uncache the logicalPlan.
>   sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = 
> true)
>   // Cache it again.
>   sparkSession.sharedState.cacheManager.cacheQuery(table, 
> Some(tableIdent.table))
> }
>   }



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org