diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 6aa0e8d8d356ee6af0a3dd51f9fe2075027a183a..cd517a98aca1c76e32ae3c3da3d6859c1bffae41 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -620,13 +620,18 @@ class Analyzer(
     private def lookupTableFromCatalog(
         u: UnresolvedRelation,
         defaultDatabase: Option[String] = None): LogicalPlan = {
+      val tableIdentWithDb = u.tableIdentifier.copy(
+        database = u.tableIdentifier.database.orElse(defaultDatabase))
       try {
-        val tableIdentWithDb = u.tableIdentifier.copy(
-          database = u.tableIdentifier.database.orElse(defaultDatabase))
         catalog.lookupRelation(tableIdentWithDb, u.alias)
       } catch {
         case _: NoSuchTableException =>
-          u.failAnalysis(s"Table or view not found: ${u.tableName}")
+          u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}")
+        // If the database is defined and that database is not found, throw an AnalysisException.
+        // Note that if the database is not defined, it is possible we are looking up a temp view.
+        case e: NoSuchDatabaseException =>
+          u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}, the " +
+            s"database ${e.db} doesn't exsits.")
       }
     }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
index 8febdcaee829b23e88970e99888404676c23adc2..f5aae60431c154672fe58b11aefc32e4b50b5f50 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
  * Thrown by a catalog when an item cannot be found. The analyzer will rethrow the exception
  * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information.
  */
-class NoSuchDatabaseException(db: String) extends AnalysisException(s"Database '$db' not found")
+class NoSuchDatabaseException(val db: String) extends AnalysisException(s"Database '$db' not found")
 
 class NoSuchTableException(db: String, table: String)
   extends AnalysisException(s"Table or view '$table' not found in database '$db'")
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
similarity index 59%
rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala
rename to sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
index 958001e8fdb2cdc153d98293ccfac7ea94acad0b..2d95cb6d64a87eacd8c4880cc5397633d2826b29 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
@@ -15,29 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.hive.execution
+package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode}
+import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
-import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType}
-import org.apache.spark.sql.hive.test.TestHiveSingleton
-import org.apache.spark.sql.test.SQLTestUtils
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+
+class SimpleSQLViewSuite extends SQLViewSuite with SharedSQLContext
 
 /**
  * A suite for testing view related functionality.
  */
-class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
-  import spark.implicits._
+abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
+  import testImplicits._
 
-  override def beforeAll(): Unit = {
+  protected override def beforeAll(): Unit = {
+    super.beforeAll()
     // Create a simple table with two columns: id and id1
     spark.range(1, 10).selectExpr("id", "id id1").write.format("json").saveAsTable("jt")
   }
 
-  override def afterAll(): Unit = {
-    spark.sql(s"DROP TABLE IF EXISTS jt")
+  protected override def afterAll(): Unit = {
+    try {
+      spark.sql(s"DROP TABLE IF EXISTS jt")
+    } finally {
+      super.afterAll()
+    }
   }
 
   test("create a permanent view on a permanent view") {
@@ -85,7 +89,7 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
 
   test("error handling: existing a table with the duplicate name when creating/altering a view") {
     withTable("tab1") {
-      sql("CREATE TABLE tab1 (id int)")
+      sql("CREATE TABLE tab1 (id int) USING parquet")
       var e = intercept[AnalysisException] {
         sql("CREATE OR REPLACE VIEW tab1 AS SELECT * FROM jt")
       }.getMessage
@@ -103,7 +107,7 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
 
   test("existing a table with the duplicate name when CREATE VIEW IF NOT EXISTS") {
     withTable("tab1") {
-      sql("CREATE TABLE tab1 (id int)")
+      sql("CREATE TABLE tab1 (id int) USING parquet")
       sql("CREATE VIEW IF NOT EXISTS tab1 AS SELECT * FROM jt")
       checkAnswer(sql("select count(*) FROM tab1"), Row(0))
     }
@@ -144,8 +148,9 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
       }.getMessage
       assert(e.contains("Inserting into an RDD-based table is not allowed"))
 
-      val testData = hiveContext.getHiveFile("data/files/employee.dat").getCanonicalPath
-      assertNoSuchTable(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE $viewName""")
+      val dataFilePath =
+        Thread.currentThread().getContextClassLoader.getResource("data/files/employee.dat")
+      assertNoSuchTable(s"""LOAD DATA LOCAL INPATH "$dataFilePath" INTO TABLE $viewName""")
       assertNoSuchTable(s"TRUNCATE TABLE $viewName")
       assertNoSuchTable(s"SHOW CREATE TABLE $viewName")
       assertNoSuchTable(s"SHOW PARTITIONS $viewName")
@@ -169,9 +174,10 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
       }.getMessage
       assert(e.contains("Inserting into an RDD-based table is not allowed"))
 
-      val testData = hiveContext.getHiveFile("data/files/employee.dat").getCanonicalPath
+      val dataFilePath =
+        Thread.currentThread().getContextClassLoader.getResource("data/files/employee.dat")
       e = intercept[AnalysisException] {
-        sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE $viewName""")
+        sql(s"""LOAD DATA LOCAL INPATH "$dataFilePath" INTO TABLE $viewName""")
       }.getMessage
       assert(e.contains(s"Target table in LOAD DATA cannot be a view: `default`.`testview`"))
 
@@ -183,10 +189,11 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
   }
 
   test("error handling: fail if the view sql itself is invalid") {
+    // A database that does not exist
+    assertInvalidReference("CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM db_not_exist234.jt")
+
     // A table that does not exist
-    intercept[AnalysisException] {
-      sql("CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM table_not_exist1345").collect()
-    }
+    assertInvalidReference("CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM table_not_exist345")
 
     // A column that does not exist
     intercept[AnalysisException] {
@@ -194,6 +201,14 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
     }
   }
 
+  private def assertInvalidReference(query: String): Unit = {
+    val e = intercept[AnalysisException] {
+      sql(query)
+    }.getMessage
+    assert(e.contains("Table or view not found"))
+  }
+
+
   test("error handling: fail if the temp view name contains the database prefix") {
     // Fully qualified table name like "database.table" is not allowed for temporary view
     val e = intercept[AnalysisException] {
@@ -210,10 +225,13 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
   }
 
   test("error handling: fail if the temp view sql itself is invalid") {
-     // A table that does not exist for temporary view
-    intercept[AnalysisException] {
-      sql("CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM table_not_exist1345")
-    }
+    // A database that does not exist
+    assertInvalidReference(
+      "CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM db_not_exist234.jt")
+
+    // A table that does not exist
+    assertInvalidReference(
+      "CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM table_not_exist1345")
 
     // A column that does not exist, for temporary view
     intercept[AnalysisException] {
@@ -233,6 +251,24 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
     }
   }
 
+  test("correctly parse a nested view") {
+    withTempDatabase { db =>
+      withView("view1", "view2", s"$db.view3") {
+        sql("CREATE VIEW view1(x, y) AS SELECT * FROM jt")
+
+        // Create a nested view in the same database.
+        sql("CREATE VIEW view2(id, id1) AS SELECT * FROM view1")
+        checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 9).map(i => Row(i, i)))
+
+        // Create a nested view in a different database.
+        activateDatabase(db) {
+          sql(s"CREATE VIEW $db.view3(id, id1) AS SELECT * FROM default.view1")
+          checkAnswer(sql("SELECT * FROM view3 ORDER BY id"), (1 to 9).map(i => Row(i, i)))
+        }
+      }
+    }
+  }
+
   test("correctly parse CREATE TEMPORARY VIEW statement") {
     withView("testView") {
       sql(
@@ -334,6 +370,20 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
     }
   }
 
+  test("correctly handle ALTER VIEW on a referenced view") {
+    withView("view1", "view2") {
+      sql("CREATE VIEW view1(x, y) AS SELECT * FROM jt")
+
+      // Create a nested view.
+      sql("CREATE VIEW view2(id, id1) AS SELECT * FROM view1")
+      checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 9).map(i => Row(i, i)))
+
+      // Alter the referenced view.
+      sql("ALTER VIEW view1 AS SELECT id AS x, id1 + 1 As y FROM jt")
+      checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 9).map(i => Row(i, i + 1)))
+    }
+  }
+
   test("should not allow ALTER VIEW AS when the view does not exist") {
     assertNoSuchTable("ALTER VIEW testView AS SELECT 1, 2")
     assertNoSuchTable("ALTER VIEW default.testView AS SELECT 1, 2")
@@ -398,7 +448,7 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
     }
   }
 
-  test("create hive view for json table") {
+  test("create view for json table") {
     // json table is not hive-compatible, make sure the new flag fix it.
     withView("testView") {
       sql("CREATE VIEW testView AS SELECT id FROM jt")
@@ -406,7 +456,7 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
     }
   }
 
-  test("create hive view for partitioned parquet table") {
+  test("create view for partitioned parquet table") {
     // partitioned parquet table is not hive-compatible, make sure the new flag fix it.
     withTable("parTable") {
       withView("testView") {
@@ -418,6 +468,22 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
     }
   }
 
+  test("create view for joined tables") {
+    // make sure the new flag can handle some complex cases like join and schema change.
+    withTable("jt1", "jt2") {
+      spark.range(1, 10).toDF("id1").write.format("json").saveAsTable("jt1")
+      spark.range(1, 10).toDF("id2").write.format("json").saveAsTable("jt2")
+      withView("testView") {
+        sql("CREATE VIEW testView AS SELECT * FROM jt1 JOIN jt2 ON id1 == id2")
+        checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i)))
+
+        val df = (1 until 10).map(i => i -> i).toDF("id1", "newCol")
+        df.write.format("json").mode(SaveMode.Overwrite).saveAsTable("jt1")
+        checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i)))
+      }
+    }
+  }
+
   test("CTE within view") {
     withView("cte_view") {
       sql("CREATE VIEW cte_view AS WITH w AS (SELECT 1 AS n) SELECT n FROM w")
@@ -427,15 +493,15 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
 
   test("Using view after switching current database") {
     withView("v") {
-      sql("CREATE VIEW v AS SELECT * FROM src")
+      sql("CREATE VIEW v AS SELECT * FROM jt")
       withTempDatabase { db =>
         activateDatabase(db) {
-          // Should look up table `src` in database `default`.
-          checkAnswer(sql("SELECT * FROM default.v"), sql("SELECT * FROM default.src"))
+          // Should look up table `jt` in database `default`.
+          checkAnswer(sql("SELECT * FROM default.v"), sql("SELECT * FROM default.jt"))
 
-          // The new `src` table shouldn't be scanned.
-          sql("CREATE TABLE src(key INT, value STRING)")
-          checkAnswer(sql("SELECT * FROM default.v"), sql("SELECT * FROM default.src"))
+          // The new `jt` table shouldn't be scanned.
+          sql("CREATE TABLE jt(key INT, value STRING) USING parquet")
+          checkAnswer(sql("SELECT * FROM default.v"), sql("SELECT * FROM default.jt"))
         }
       }
     }
@@ -452,311 +518,103 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
     }
   }
 
-  test("create hive view for joined tables") {
-    // make sure the new flag can handle some complex cases like join and schema change.
-    withTable("jt1", "jt2") {
-      spark.range(1, 10).toDF("id1").write.format("json").saveAsTable("jt1")
-      spark.range(1, 10).toDF("id2").write.format("json").saveAsTable("jt2")
-      sql("CREATE VIEW testView AS SELECT * FROM jt1 JOIN jt2 ON id1 == id2")
-      checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i)))
-
-      val df = (1 until 10).map(i => i -> i).toDF("id1", "newCol")
-      df.write.format("json").mode(SaveMode.Overwrite).saveAsTable("jt1")
-      checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i)))
-
-      sql("DROP VIEW testView")
-    }
-  }
-
-  test("SPARK-14933 - create view from hive parquet table") {
-    withTable("t_part") {
-      withView("v_part") {
-        spark.sql("create table t_part stored as parquet as select 1 as a, 2 as b")
-        spark.sql("create view v_part as select * from t_part")
-        checkAnswer(
-          sql("select * from t_part"),
-          sql("select * from v_part"))
-      }
-    }
-  }
-
-  test("SPARK-14933 - create view from hive orc table") {
-    withTable("t_orc") {
-      withView("v_orc") {
-        spark.sql("create table t_orc stored as orc as select 1 as a, 2 as b")
-        spark.sql("create view v_orc as select * from t_orc")
-        checkAnswer(
-          sql("select * from t_orc"),
-          sql("select * from v_orc"))
-      }
-    }
-  }
-
-  test("create a permanent/temp view using a hive, built-in, and permanent user function") {
-    val permanentFuncName = "myUpper"
-    val permanentFuncClass =
-      classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper].getCanonicalName
-    val builtInFuncNameInLowerCase = "abs"
-    val builtInFuncNameInMixedCase = "aBs"
-    val hiveFuncName = "histogram_numeric"
-
-    withUserDefinedFunction(permanentFuncName -> false) {
-      sql(s"CREATE FUNCTION $permanentFuncName AS '$permanentFuncClass'")
-      withTable("tab1") {
-        (1 to 10).map(i => (s"$i", i)).toDF("str", "id").write.saveAsTable("tab1")
-        Seq("VIEW", "TEMPORARY VIEW").foreach { viewMode =>
-          withView("view1") {
-            sql(
-              s"""
-                 |CREATE $viewMode view1
-                 |AS SELECT
-                 |$permanentFuncName(str),
-                 |$builtInFuncNameInLowerCase(id),
-                 |$builtInFuncNameInMixedCase(id) as aBs,
-                 |$hiveFuncName(id, 5) over()
-                 |FROM tab1
-               """.stripMargin)
-            checkAnswer(sql("select count(*) FROM view1"), Row(10))
-          }
-        }
-      }
-    }
-  }
-
-  test("create a permanent/temp view using a temporary function") {
-    val tempFunctionName = "temp"
-    val functionClass =
-      classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper].getCanonicalName
-    withUserDefinedFunction(tempFunctionName -> true) {
-      sql(s"CREATE TEMPORARY FUNCTION $tempFunctionName AS '$functionClass'")
-      withView("view1", "tempView1") {
-        withTable("tab1") {
-          (1 to 10).map(i => s"$i").toDF("id").write.saveAsTable("tab1")
-
-          // temporary view
-          sql(s"CREATE TEMPORARY VIEW tempView1 AS SELECT $tempFunctionName(id) from tab1")
-          checkAnswer(sql("select count(*) FROM tempView1"), Row(10))
-
-          // permanent view
-          val e = intercept[AnalysisException] {
-            sql(s"CREATE VIEW view1 AS SELECT $tempFunctionName(id) from tab1")
-          }.getMessage
-          assert(e.contains("Not allowed to create a permanent view `view1` by referencing " +
-            s"a temporary function `$tempFunctionName`"))
-        }
-      }
-    }
-  }
-
-  test("correctly resolve a nested view") {
-    withTempDatabase { db =>
-      withView(s"$db.view1", s"$db.view2") {
-        val view1 = CatalogTable(
-          identifier = TableIdentifier("view1", Some(db)),
-          tableType = CatalogTableType.VIEW,
-          storage = CatalogStorageFormat.empty,
-          schema = new StructType().add("x", "long").add("y", "long"),
-          viewText = Some("SELECT * FROM jt"),
-          properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default",
-            CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2",
-            s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}0" -> "id",
-            s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}1" -> "id1"))
-        val view2 = CatalogTable(
-          identifier = TableIdentifier("view2", Some(db)),
-          tableType = CatalogTableType.VIEW,
-          storage = CatalogStorageFormat.empty,
-          schema = new StructType().add("id", "long").add("id1", "long"),
-          viewText = Some("SELECT * FROM view1"),
-          properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> db,
-            CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2",
-            s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}0" -> "x",
-            s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}1" -> "y"))
-        activateDatabase(db) {
-          hiveContext.sessionState.catalog.createTable(view1, ignoreIfExists = false)
-          hiveContext.sessionState.catalog.createTable(view2, ignoreIfExists = false)
-          checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 9).map(i => Row(i, i)))
-        }
-      }
-    }
-  }
-
-  test("correctly resolve a view with CTE") {
-    withView("cte_view") {
-      val cte_view = CatalogTable(
-        identifier = TableIdentifier("cte_view"),
-        tableType = CatalogTableType.VIEW,
-        storage = CatalogStorageFormat.empty,
-        schema = new StructType().add("n", "int"),
-        viewText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM w"),
-        properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default",
-          CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "1",
-          s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}0" -> "n"))
-      hiveContext.sessionState.catalog.createTable(cte_view, ignoreIfExists = false)
-      checkAnswer(sql("SELECT * FROM cte_view"), Row(1))
-    }
-  }
-
-  test("correctly resolve a view in a self join") {
-    withView("join_view") {
-      val join_view = CatalogTable(
-        identifier = TableIdentifier("join_view"),
-        tableType = CatalogTableType.VIEW,
-        storage = CatalogStorageFormat.empty,
-        schema = new StructType().add("id", "long").add("id1", "long"),
-        viewText = Some("SELECT * FROM jt"),
-        properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default",
-          CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2",
-          s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}0" -> "id",
-          s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}1" -> "id1"))
-      hiveContext.sessionState.catalog.createTable(join_view, ignoreIfExists = false)
-      checkAnswer(
-        sql("SELECT * FROM join_view t1 JOIN join_view t2 ON t1.id = t2.id ORDER BY t1.id"),
-        (1 to 9).map(i => Row(i, i, i, i)))
-    }
-  }
-
-  private def assertInvalidReference(query: String): Unit = {
-    val e = intercept[AnalysisException] {
-      sql(query)
-    }.getMessage
-    assert(e.contains("Table or view not found"))
-  }
-
   test("error handling: fail if the referenced table or view is invalid") {
     withView("view1", "view2", "view3") {
       // Fail if the referenced table is defined in a invalid database.
-      val view1 = CatalogTable(
-        identifier = TableIdentifier("view1"),
-        tableType = CatalogTableType.VIEW,
-        storage = CatalogStorageFormat.empty,
-        schema = new StructType().add("id", "long").add("id1", "long"),
-        viewText = Some("SELECT * FROM invalid_db.jt"),
-        properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default",
-          CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2",
-          s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}0" -> "id",
-          s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}1" -> "id1"))
-      hiveContext.sessionState.catalog.createTable(view1, ignoreIfExists = false)
+      withTempDatabase { db =>
+        withTable(s"$db.table1") {
+          activateDatabase(db) {
+            sql("CREATE TABLE table1(a int, b string) USING parquet")
+            sql("CREATE VIEW default.view1 AS SELECT * FROM table1")
+          }
+        }
+      }
       assertInvalidReference("SELECT * FROM view1")
 
       // Fail if the referenced table is invalid.
-      val view2 = CatalogTable(
-        identifier = TableIdentifier("view2"),
-        tableType = CatalogTableType.VIEW,
-        storage = CatalogStorageFormat.empty,
-        schema = new StructType().add("id", "long").add("id1", "long"),
-        viewText = Some("SELECT * FROM invalid_table"),
-        properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default",
-          CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2",
-          s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}0" -> "id",
-          s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}1" -> "id1"))
-      hiveContext.sessionState.catalog.createTable(view2, ignoreIfExists = false)
+      withTable("table2") {
+        sql("CREATE TABLE table2(a int, b string) USING parquet")
+        sql("CREATE VIEW view2 AS SELECT * FROM table2")
+      }
       assertInvalidReference("SELECT * FROM view2")
 
       // Fail if the referenced view is invalid.
-      val view3 = CatalogTable(
-        identifier = TableIdentifier("view3"),
-        tableType = CatalogTableType.VIEW,
-        storage = CatalogStorageFormat.empty,
-        schema = new StructType().add("id", "long").add("id1", "long"),
-        viewText = Some("SELECT * FROM view2"),
-        properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default",
-          CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2",
-          s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}0" -> "id",
-          s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}1" -> "id1"))
-      hiveContext.sessionState.catalog.createTable(view3, ignoreIfExists = false)
+      withView("testView") {
+        sql("CREATE VIEW testView AS SELECT * FROM jt")
+        sql("CREATE VIEW view3 AS SELECT * FROM testView")
+      }
       assertInvalidReference("SELECT * FROM view3")
     }
   }
 
-  test("make sure we can resolve view created by old version of Spark") {
-    withTable("hive_table") {
-      withView("old_view") {
-        spark.sql("CREATE TABLE hive_table AS SELECT 1 AS a, 2 AS b")
-        // The views defined by older versions of Spark(before 2.2) will have empty view default
-        // database name, and all the relations referenced in the viewText will have database part
-        // defined.
-        val view = CatalogTable(
-          identifier = TableIdentifier("old_view"),
-          tableType = CatalogTableType.VIEW,
-          storage = CatalogStorageFormat.empty,
-          schema = new StructType().add("a", "int").add("b", "int"),
-          viewText = Some("SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b` FROM (SELECT " +
-            "`gen_attr_0`, `gen_attr_1` FROM (SELECT `a` AS `gen_attr_0`, `b` AS " +
-            "`gen_attr_1` FROM hive_table) AS gen_subquery_0) AS hive_table")
-        )
-        hiveContext.sessionState.catalog.createTable(view, ignoreIfExists = false)
-        val df = sql("SELECT * FROM old_view")
-        // Check the output rows.
-        checkAnswer(df, Row(1, 2))
-        // Check the output schema.
-        assert(df.schema.sameType(view.schema))
-      }
+  test("correctly resolve a view in a self join") {
+    withView("testView") {
+      sql("CREATE VIEW testView AS SELECT * FROM jt")
+      checkAnswer(
+        sql("SELECT * FROM testView t1 JOIN testView t2 ON t1.id = t2.id ORDER BY t1.id"),
+        (1 to 9).map(i => Row(i, i, i, i)))
     }
   }
 
-  test("resolve a view with custom column names") {
-    withTable("testTable") {
-      spark.range(1, 10).selectExpr("id", "id + 1 id1").write.saveAsTable("testTable")
-      withView("testView") {
-        val testView = CatalogTable(
-          identifier = TableIdentifier("testView"),
-          tableType = CatalogTableType.VIEW,
-          storage = CatalogStorageFormat.empty,
-          schema = new StructType().add("x", "long").add("y", "long"),
-          viewText = Some("SELECT * FROM testTable"),
-          properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default",
-            CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2",
-            s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}0" -> "id",
-            s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}1" -> "id1"))
-        hiveContext.sessionState.catalog.createTable(testView, ignoreIfExists = false)
+  test("correctly handle a view with custom column names") {
+    withTable("tab1") {
+      spark.range(1, 10).selectExpr("id", "id + 1 id1").write.saveAsTable("tab1")
+      withView("testView", "testView2") {
+        sql("CREATE VIEW testView(x, y) AS SELECT * FROM tab1")
 
         // Correctly resolve a view with custom column names.
         checkAnswer(sql("SELECT * FROM testView ORDER BY x"), (1 to 9).map(i => Row(i, i + 1)))
 
+        // Throw an AnalysisException if the number of columns don't match up.
+        val e = intercept[AnalysisException] {
+          sql("CREATE VIEW testView2(x, y, z) AS SELECT * FROM tab1")
+        }.getMessage
+        assert(e.contains("The number of columns produced by the SELECT clause (num: `2`) does " +
+          "not match the number of column names specified by CREATE VIEW (num: `3`)."))
+
         // Correctly resolve a view when the referenced table schema changes.
         spark.range(1, 10).selectExpr("id", "id + id dummy", "id + 1 id1")
-          .write.mode(SaveMode.Overwrite).saveAsTable("testTable")
+          .write.mode(SaveMode.Overwrite).saveAsTable("tab1")
         checkAnswer(sql("SELECT * FROM testView ORDER BY x"), (1 to 9).map(i => Row(i, i + 1)))
 
         // Throw an AnalysisException if the column name is not found.
         spark.range(1, 10).selectExpr("id", "id + 1 dummy")
-          .write.mode(SaveMode.Overwrite).saveAsTable("testTable")
+          .write.mode(SaveMode.Overwrite).saveAsTable("tab1")
         intercept[AnalysisException](sql("SELECT * FROM testView"))
       }
     }
   }
 
   test("resolve a view when the dataTypes of referenced table columns changed") {
-    withTable("testTable") {
-      spark.range(1, 10).selectExpr("id", "id + 1 id1").write.saveAsTable("testTable")
+    withTable("tab1") {
+      spark.range(1, 10).selectExpr("id", "id + 1 id1").write.saveAsTable("tab1")
       withView("testView") {
-        val testView = CatalogTable(
-          identifier = TableIdentifier("testView"),
-          tableType = CatalogTableType.VIEW,
-          storage = CatalogStorageFormat.empty,
-          schema = new StructType().add("id", "long").add("id1", "long"),
-          viewText = Some("SELECT * FROM testTable"),
-          properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default",
-            CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2",
-            s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}0" -> "id",
-            s"${CatalogTable.VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX}1" -> "id1"))
-        hiveContext.sessionState.catalog.createTable(testView, ignoreIfExists = false)
+        sql("CREATE VIEW testView AS SELECT * FROM tab1")
 
         // Allow casting from IntegerType to LongType
         val df = (1 until 10).map(i => (i, i + 1)).toDF("id", "id1")
-        df.write.format("json").mode(SaveMode.Overwrite).saveAsTable("testTable")
+        df.write.format("json").mode(SaveMode.Overwrite).saveAsTable("tab1")
         checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i + 1)))
 
         // Casting from DoubleType to LongType might truncate, throw an AnalysisException.
         val df2 = (1 until 10).map(i => (i.toDouble, i.toDouble)).toDF("id", "id1")
-        df2.write.format("json").mode(SaveMode.Overwrite).saveAsTable("testTable")
+        df2.write.format("json").mode(SaveMode.Overwrite).saveAsTable("tab1")
         intercept[AnalysisException](sql("SELECT * FROM testView"))
 
         // Can't cast from ArrayType to LongType, throw an AnalysisException.
         val df3 = (1 until 10).map(i => (i, Seq(i))).toDF("id", "id1")
-        df3.write.format("json").mode(SaveMode.Overwrite).saveAsTable("testTable")
+        df3.write.format("json").mode(SaveMode.Overwrite).saveAsTable("tab1")
         intercept[AnalysisException](sql("SELECT * FROM testView"))
       }
     }
   }
+
+  // TODO: Check for cyclic view references on ALTER VIEW.
+  ignore("correctly handle a cyclic view reference") {
+    withView("view1", "view2") {
+      sql("CREATE VIEW view1 AS SELECT * FROM jt")
+      sql("CREATE VIEW view2 AS SELECT * FROM view1")
+      intercept[AnalysisException](sql("ALTER VIEW view1 AS SELECT * FROM view2"))
+    }
+  }
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala
index 36dc2368fd9586a97537d8815fea0d900794293c..e122b39f6fc40be5564e3dde04c29230d7f0626a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala
@@ -70,13 +70,10 @@ trait SharedSQLContext extends SQLTestUtils with BeforeAndAfterEach {
    * Stop the underlying [[org.apache.spark.SparkContext]], if any.
    */
   protected override def afterAll(): Unit = {
-    try {
-      if (_spark != null) {
-        _spark.stop()
-        _spark = null
-      }
-    } finally {
-      super.afterAll()
+    super.afterAll()
+    if (_spark != null) {
+      _spark.stop()
+      _spark = null
     }
   }
 
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5afb37b382e65c046ca342e891dab0965afb3410
--- /dev/null
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive.execution
+
+import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType}
+import org.apache.spark.sql.execution.SQLViewSuite
+import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * A test suite for Hive view related functionality.
+ */
+class HiveSQLViewSuite extends SQLViewSuite with TestHiveSingleton {
+  protected override val spark: SparkSession = TestHive.sparkSession
+
+  import testImplicits._
+
+  test("create a permanent/temp view using a hive, built-in, and permanent user function") {
+    val permanentFuncName = "myUpper"
+    val permanentFuncClass =
+      classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper].getCanonicalName
+    val builtInFuncNameInLowerCase = "abs"
+    val builtInFuncNameInMixedCase = "aBs"
+    val hiveFuncName = "histogram_numeric"
+
+    withUserDefinedFunction(permanentFuncName -> false) {
+      sql(s"CREATE FUNCTION $permanentFuncName AS '$permanentFuncClass'")
+      withTable("tab1") {
+        (1 to 10).map(i => (s"$i", i)).toDF("str", "id").write.saveAsTable("tab1")
+        Seq("VIEW", "TEMPORARY VIEW").foreach { viewMode =>
+          withView("view1") {
+            sql(
+              s"""
+                 |CREATE $viewMode view1
+                 |AS SELECT
+                 |$permanentFuncName(str),
+                 |$builtInFuncNameInLowerCase(id),
+                 |$builtInFuncNameInMixedCase(id) as aBs,
+                 |$hiveFuncName(id, 5) over()
+                 |FROM tab1
+               """.stripMargin)
+            checkAnswer(sql("select count(*) FROM view1"), Row(10))
+          }
+        }
+      }
+    }
+  }
+
+  test("create a permanent/temp view using a temporary function") {
+    val tempFunctionName = "temp"
+    val functionClass =
+      classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper].getCanonicalName
+    withUserDefinedFunction(tempFunctionName -> true) {
+      sql(s"CREATE TEMPORARY FUNCTION $tempFunctionName AS '$functionClass'")
+      withView("view1", "tempView1") {
+        withTable("tab1") {
+          (1 to 10).map(i => s"$i").toDF("id").write.saveAsTable("tab1")
+
+          // temporary view
+          sql(s"CREATE TEMPORARY VIEW tempView1 AS SELECT $tempFunctionName(id) from tab1")
+          checkAnswer(sql("select count(*) FROM tempView1"), Row(10))
+
+          // permanent view
+          val e = intercept[AnalysisException] {
+            sql(s"CREATE VIEW view1 AS SELECT $tempFunctionName(id) from tab1")
+          }.getMessage
+          assert(e.contains("Not allowed to create a permanent view `view1` by referencing " +
+            s"a temporary function `$tempFunctionName`"))
+        }
+      }
+    }
+  }
+
+  test("SPARK-14933 - create view from hive parquet table") {
+    withTable("t_part") {
+      withView("v_part") {
+        spark.sql("create table t_part stored as parquet as select 1 as a, 2 as b")
+        spark.sql("create view v_part as select * from t_part")
+        checkAnswer(
+          sql("select * from t_part"),
+          sql("select * from v_part"))
+      }
+    }
+  }
+
+  test("SPARK-14933 - create view from hive orc table") {
+    withTable("t_orc") {
+      withView("v_orc") {
+        spark.sql("create table t_orc stored as orc as select 1 as a, 2 as b")
+        spark.sql("create view v_orc as select * from t_orc")
+        checkAnswer(
+          sql("select * from t_orc"),
+          sql("select * from v_orc"))
+      }
+    }
+  }
+
+  test("make sure we can resolve view created by old version of Spark") {
+    withTable("hive_table") {
+      withView("old_view") {
+        spark.sql("CREATE TABLE hive_table AS SELECT 1 AS a, 2 AS b")
+        // The views defined by older versions of Spark(before 2.2) will have empty view default
+        // database name, and all the relations referenced in the viewText will have database part
+        // defined.
+        val view = CatalogTable(
+          identifier = TableIdentifier("old_view"),
+          tableType = CatalogTableType.VIEW,
+          storage = CatalogStorageFormat.empty,
+          schema = new StructType().add("a", "int").add("b", "int"),
+          viewText = Some("SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b` FROM (SELECT " +
+            "`gen_attr_0`, `gen_attr_1` FROM (SELECT `a` AS `gen_attr_0`, `b` AS " +
+            "`gen_attr_1` FROM hive_table) AS gen_subquery_0) AS hive_table")
+        )
+        hiveContext.sessionState.catalog.createTable(view, ignoreIfExists = false)
+        val df = sql("SELECT * FROM old_view")
+        // Check the output rows.
+        checkAnswer(df, Row(1, 2))
+        // Check the output schema.
+        assert(df.schema.sameType(view.schema))
+      }
+    }
+  }
+}