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

Laszlo Rigo commented on SPARK-26727:
-------------------------------------

For me these calls still seem to be asynchronous:
{noformat}
var i=0
try{
  while(true){
    println(i)
    spark.sql("DROP VIEW IF EXISTS testSparkReplace")
    spark.sql("CREATE VIEW testSparkReplace as SELECT dummy FROM ae_dual")
    while(!spark.catalog.tableExists("testSparkReplace")) {}
    i=i+1
  }
}catch{
  case e: Exception => e.printStackTrace()
}{noformat}
This script failed with the same exception when the value of 'i' was 183:
{noformat}
org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
AlreadyExistsException(message:Table testsparkreplace already exists);
at 
org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:106)
at 
org.apache.spark.sql.hive.HiveExternalCatalog.createTable(HiveExternalCatalog.scala:236)
at 
org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.createTable(ExternalCatalogWithListener.scala:94)
at 
org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTable(SessionCatalog.scala:319)
at org.apache.spark.sql.execution.command.CreateViewCommand.run(views.scala:175)
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:195)
at org.apache.spark.sql.Dataset$$anonfun$6.apply(Dataset.scala:195)
at org.apache.spark.sql.Dataset$$anonfun$53.apply(Dataset.scala:3365)
at 
org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3364)
at org.apache.spark.sql.Dataset.<init>(Dataset.scala:195)
at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:80)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:642)
at $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.liftedTree1$1(<console>:30)
at $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)
at $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:41)
at $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:43)
at $line17.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:45)
at $line17.$read$$iw$$iw$$iw$$iw.<init>(<console>:47)
at $line17.$read$$iw$$iw$$iw.<init>(<console>:49)
at $line17.$read$$iw$$iw.<init>(<console>:51)
at $line17.$read$$iw.<init>(<console>:53)
at $line17.$read.<init>(<console>:55)
at $line17.$read$.<init>(<console>:59)
at $line17.$read$.<clinit>(<console>)
at $line17.$eval$.$print$lzycompute(<console>:7)
at $line17.$eval$.$print(<console>:6)
at $line17.$eval.$print(<console>)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:793)
at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:1054)
at 
scala.tools.nsc.interpreter.IMain$WrappedRequest$$anonfun$loadAndRunReq$1.apply(IMain.scala:645)
at 
scala.tools.nsc.interpreter.IMain$WrappedRequest$$anonfun$loadAndRunReq$1.apply(IMain.scala:644)
at 
scala.reflect.internal.util.ScalaClassLoader$class.asContext(ScalaClassLoader.scala:31)
at 
scala.reflect.internal.util.AbstractFileClassLoader.asContext(AbstractFileClassLoader.scala:19)
at 
scala.tools.nsc.interpreter.IMain$WrappedRequest.loadAndRunReq(IMain.scala:644)
at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:576)
at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:572)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:819)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:837)
at scala.tools.nsc.interpreter.ILoop.command(ILoop.scala:691)
at scala.tools.nsc.interpreter.ILoop.processLine(ILoop.scala:404)
at scala.tools.nsc.interpreter.ILoop.loop(ILoop.scala:425)
at 
org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply$mcZ$sp(SparkILoop.scala:285)
at org.apache.spark.repl.SparkILoop.runClosure(SparkILoop.scala:159)
at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:182)
at org.apache.spark.repl.Main$.doMain(Main.scala:78)
at org.apache.spark.repl.Main$.main(Main.scala:58)
at org.apache.spark.repl.Main.main(Main.scala)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
at 
org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:849)
at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:167)
at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:195)
at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:924)
at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:933)
at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
AlreadyExistsException(message:Table testsparkreplace already exists)
at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:717)
at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$createTable$1.apply$mcV$sp(HiveClientImpl.scala:481)
at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$createTable$1.apply(HiveClientImpl.scala:479)
at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$createTable$1.apply(HiveClientImpl.scala:479)
at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:275)
at 
org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:213)
at 
org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:212)
at 
org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:258)
at 
org.apache.spark.sql.hive.client.HiveClientImpl.createTable(HiveClientImpl.scala:479)
at 
org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$createTable$1.apply$mcV$sp(HiveExternalCatalog.scala:278)
at 
org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$createTable$1.apply(HiveExternalCatalog.scala:236)
at 
org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$createTable$1.apply(HiveExternalCatalog.scala:236)
at 
org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97)
... 77 more
Caused by: AlreadyExistsException(message:Table testsparkreplace already exists)
at 
org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$create_table_with_environment_context_result$create_table_with_environment_context_resultStandardScheme.read(ThriftHiveMetastore.java:29965)
at 
org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$create_table_with_environment_context_result$create_table_with_environment_context_resultStandardScheme.read(ThriftHiveMetastore.java:29951)
at 
org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$create_table_with_environment_context_result.read(ThriftHiveMetastore.java:29877)
at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:86)
at 
org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_create_table_with_environment_context(ThriftHiveMetastore.java:1075)
at 
org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.create_table_with_environment_context(ThriftHiveMetastore.java:1061)
at 
org.apache.hadoop.hive.metastore.HiveMetaStoreClient.create_table_with_environment_context(HiveMetaStoreClient.java:2050)
at 
org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.create_table_with_environment_context(SessionHiveMetaStoreClient.java:97)
at 
org.apache.hadoop.hive.metastore.HiveMetaStoreClient.createTable(HiveMetaStoreClient.java:669)
at 
org.apache.hadoop.hive.metastore.HiveMetaStoreClient.createTable(HiveMetaStoreClient.java:657)
at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:156)
at com.sun.proxy.$Proxy14.createTable(Unknown Source)
at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:714)
... 89 more

{noformat}
It seems that the drop table query does not have effect yet when the create 
table is called already. Nobody/nothing else is using this view, it was created 
for this test.

> CREATE OR REPLACE VIEW query fails with TableAlreadyExistsException
> -------------------------------------------------------------------
>
>                 Key: SPARK-26727
>                 URL: https://issues.apache.org/jira/browse/SPARK-26727
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.4.0
>            Reporter: Srinivas Yarra
>            Priority: Major
>
> We experienced that sometimes the Hive query "CREATE OR REPLACE VIEW <view 
> name> AS SELECT <columns> FROM <table>" fails with the following exception:
> {code:java}
> // code placeholder
> org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException: Table or 
> view '<view name>' already exists in database 'default'; at 
> org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTable(SessionCatalog.scala:314)
>  at 
> org.apache.spark.sql.execution.command.CreateViewCommand.run(views.scala:165) 
> 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:195) at 
> org.apache.spark.sql.Dataset$$anonfun$6.apply(Dataset.scala:195) at 
> org.apache.spark.sql.Dataset$$anonfun$53.apply(Dataset.scala:3365) at 
> org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
>  at 
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
>  at 
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
>  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3364) at 
> org.apache.spark.sql.Dataset.<init>(Dataset.scala:195) at 
> org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:80) at 
> org.apache.spark.sql.SparkSession.sql(SparkSession.scala:642) ... 49 elided
> {code}
> {code}
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res1: org.apache.spark.sql.DataFrame = []
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res2: org.apache.spark.sql.DataFrame = [] 
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res3: org.apache.spark.sql.DataFrame = [] 
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res4: org.apache.spark.sql.DataFrame = [] 
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res5: org.apache.spark.sql.DataFrame = [] 
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res6: org.apache.spark.sql.DataFrame = [] 
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res7: org.apache.spark.sql.DataFrame = []
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res8: org.apache.spark.sql.DataFrame = [] 
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res9: org.apache.spark.sql.DataFrame = [] 
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res10: org.apache.spark.sql.DataFrame = [] 
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") res11: org.apache.spark.sql.DataFrame = [] 
> scala> spark.sql("CREATE OR REPLACE VIEW testSparkReplace as SELECT dummy 
> FROM ae_dual") 
> org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException: Table or 
> view 'testsparkreplace' already exists in database 'default'; at 
> org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$createTable$1.apply$mcV$sp(HiveExternalCatalog.scala:246)
>  at 
> org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$createTable$1.apply(HiveExternalCatalog.scala:236)
>  at 
> org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$createTable$1.apply(HiveExternalCatalog.scala:236)
>  at 
> org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97)
>  at 
> org.apache.spark.sql.hive.HiveExternalCatalog.createTable(HiveExternalCatalog.scala:236)
>  at 
> org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.createTable(ExternalCatalogWithListener.scala:94)
>  at 
> org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTable(SessionCatalog.scala:319)
>  at 
> org.apache.spark.sql.execution.command.CreateViewCommand.run(views.scala:165) 
> 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:195) at 
> org.apache.spark.sql.Dataset$$anonfun$6.apply(Dataset.scala:195) at 
> org.apache.spark.sql.Dataset$$anonfun$53.apply(Dataset.scala:3365) at 
> org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
>  at 
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
>  at 
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
>  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3364) at 
> org.apache.spark.sql.Dataset.<init>(Dataset.scala:195) at 
> org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:80) at
> org.apache.spark.sql.SparkSession.sql(SparkSession.scala:642) ... 49 elided 
> scala> spark.catalog.tableExists("testSparkReplace") res13: Boolean = false 
> scala>
> {code}
>  
> As we can see, it does not happen for the first 11 executions and then the 
> exception is thrown  and then the table does not exist, so it is dropped, but 
> not created again



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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

Reply via email to