diff options
author | Sean Zhong <seanzhong@databricks.com> | 2016-05-18 09:01:59 +0800 |
---|---|---|
committer | Cheng Lian <lian@databricks.com> | 2016-05-18 09:01:59 +0800 |
commit | 25b315e6cad7c27b62dcaa2c194293c1115fdfb3 (patch) | |
tree | cfeebcaf553d78ca80a70f7139a765e7759f0410 /sql/core | |
parent | b674e67c22bf663334e537e35787c00533adbb04 (diff) | |
download | spark-25b315e6cad7c27b62dcaa2c194293c1115fdfb3.tar.gz spark-25b315e6cad7c27b62dcaa2c194293c1115fdfb3.tar.bz2 spark-25b315e6cad7c27b62dcaa2c194293c1115fdfb3.zip |
[SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable
## What changes were proposed in this pull request?
Update the unit test code, examples, and documents to remove calls to deprecated method `dataset.registerTempTable`.
## How was this patch tested?
This PR only changes the unit test code, examples, and comments. It should be safe.
This is a follow up of PR https://github.com/apache/spark/pull/12945 which was merged.
Author: Sean Zhong <seanzhong@databricks.com>
Closes #13098 from clockfly/spark-15171-remove-deprecation.
Diffstat (limited to 'sql/core')
31 files changed, 217 insertions, 213 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 44511885a7..a3e2b49556 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -376,7 +376,7 @@ class SQLContext private[sql]( * // |-- name: string (nullable = false) * // |-- age: integer (nullable = true) * - * dataFrame.registerTempTable("people") + * dataFrame.createOrReplaceTempView("people") * sqlContext.sql("select name from people").collect.foreach(println) * }}} * diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java index f2ae40e644..573d0e3594 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java @@ -108,7 +108,7 @@ public class JavaApplySchemaSuite implements Serializable { StructType schema = DataTypes.createStructType(fields); Dataset<Row> df = spark.createDataFrame(rowRDD, schema); - df.registerTempTable("people"); + df.createOrReplaceTempView("people"); List<Row> actual = spark.sql("SELECT * FROM people").collectAsList(); List<Row> expected = new ArrayList<>(2); @@ -144,7 +144,7 @@ public class JavaApplySchemaSuite implements Serializable { StructType schema = DataTypes.createStructType(fields); Dataset<Row> df = spark.createDataFrame(rowRDD, schema); - df.registerTempTable("people"); + df.createOrReplaceTempView("people"); List<String> actual = spark.sql("SELECT * FROM people").toJavaRDD() .map(new Function<Row, String>() { @Override @@ -202,14 +202,14 @@ public class JavaApplySchemaSuite implements Serializable { Dataset<Row> df1 = spark.read().json(jsonRDD); StructType actualSchema1 = df1.schema(); Assert.assertEquals(expectedSchema, actualSchema1); - df1.registerTempTable("jsonTable1"); + df1.createOrReplaceTempView("jsonTable1"); List<Row> actual1 = spark.sql("select * from jsonTable1").collectAsList(); Assert.assertEquals(expectedResult, actual1); Dataset<Row> df2 = spark.read().schema(expectedSchema).json(jsonRDD); StructType actualSchema2 = df2.schema(); Assert.assertEquals(expectedSchema, actualSchema2); - df2.registerTempTable("jsonTable2"); + df2.createOrReplaceTempView("jsonTable2"); List<Row> actual2 = spark.sql("select * from jsonTable2").collectAsList(); Assert.assertEquals(expectedResult, actual2); } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java index d0435e4d43..9840bc46f9 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java @@ -72,7 +72,7 @@ public class JavaSaveLoadSuite { } JavaRDD<String> rdd = jsc.parallelize(jsonObjects); df = spark.read().json(rdd); - df.registerTempTable("jsonTable"); + df.createOrReplaceTempView("jsonTable"); } @After diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 6d8de80a11..1c96bdc05c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -71,7 +71,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("cache temp table") { - testData.select('key).registerTempTable("tempTable") + testData.select('key).createOrReplaceTempView("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) spark.catalog.cacheTable("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable")) @@ -99,8 +99,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("uncaching temp table") { - testData.select('key).registerTempTable("tempTable1") - testData.select('key).registerTempTable("tempTable2") + testData.select('key).createOrReplaceTempView("tempTable1") + testData.select('key).createOrReplaceTempView("tempTable2") spark.catalog.cacheTable("tempTable1") assertCached(sql("SELECT COUNT(*) FROM tempTable1")) @@ -116,7 +116,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext test("too big for memory") { val data = "*" * 1000 sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF() - .registerTempTable("bigData") + .createOrReplaceTempView("bigData") spark.table("bigData").persist(StorageLevel.MEMORY_AND_DISK) assert(spark.table("bigData").count() === 200000L) spark.table("bigData").unpersist(blocking = true) @@ -191,7 +191,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("SELECT star from cached table") { - sql("SELECT * FROM testData").registerTempTable("selectStar") + sql("SELECT * FROM testData").createOrReplaceTempView("selectStar") spark.catalog.cacheTable("selectStar") checkAnswer( sql("SELECT * FROM selectStar WHERE key = 1"), @@ -286,15 +286,15 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("Drops temporary table") { - testData.select('key).registerTempTable("t1") + testData.select('key).createOrReplaceTempView("t1") spark.table("t1") spark.catalog.dropTempView("t1") intercept[AnalysisException](spark.table("t1")) } test("Drops cached temporary table") { - testData.select('key).registerTempTable("t1") - testData.select('key).registerTempTable("t2") + testData.select('key).createOrReplaceTempView("t1") + testData.select('key).createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") assert(spark.catalog.isCached("t1")) @@ -306,15 +306,15 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("Clear all cache") { - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") spark.catalog.clearCache() assert(spark.cacheManager.isEmpty) - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") sql("Clear CACHE") @@ -322,8 +322,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("Clear accumulators when uncacheTable to prevent memory leaking") { - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -350,7 +350,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext test("SPARK-10327 Cache Table is not working while subquery has alias in its project list") { sparkContext.parallelize((1, 1) :: (2, 2) :: Nil) - .toDF("key", "value").selectExpr("key", "value", "key+1").registerTempTable("abc") + .toDF("key", "value").selectExpr("key", "value", "key+1").createOrReplaceTempView("abc") spark.catalog.cacheTable("abc") val sparkPlan = sql( @@ -371,9 +371,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { val table3x = testData.union(testData).union(testData) - table3x.registerTempTable("testData3x") + table3x.createOrReplaceTempView("testData3x") - sql("SELECT key, value FROM testData3x ORDER BY key").registerTempTable("orderedTable") + sql("SELECT key, value FROM testData3x ORDER BY key").createOrReplaceTempView("orderedTable") spark.catalog.cacheTable("orderedTable") assertCached(spark.table("orderedTable")) // Should not have an exchange as the query is already sorted on the group by key. @@ -388,8 +388,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // different number of partitions. for (numPartitions <- 1 until 10 by 4) { withTempTable("t1", "t2") { - testData.repartition(numPartitions, $"key").registerTempTable("t1") - testData2.repartition(numPartitions, $"a").registerTempTable("t2") + testData.repartition(numPartitions, $"key").createOrReplaceTempView("t1") + testData2.repartition(numPartitions, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -410,8 +410,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // Distribute the tables into non-matching number of partitions. Need to shuffle one side. withTempTable("t1", "t2") { - testData.repartition(6, $"key").registerTempTable("t1") - testData2.repartition(3, $"a").registerTempTable("t2") + testData.repartition(6, $"key").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -427,8 +427,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // One side of join is not partitioned in the desired way. Need to shuffle one side. withTempTable("t1", "t2") { - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(6, $"a").registerTempTable("t2") + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(6, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -443,8 +443,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } withTempTable("t1", "t2") { - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(12, $"a").registerTempTable("t2") + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(12, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -462,8 +462,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // the side that has already partitioned is smaller than the side that is not partitioned, // we shuffle both side. withTempTable("t1", "t2") { - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(3, $"a").registerTempTable("t2") + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -479,7 +479,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // repartition's column ordering is different from group by column ordering. // But they use the same set of columns. withTempTable("t1") { - testData.repartition(6, $"value", $"key").registerTempTable("t1") + testData.repartition(6, $"value", $"key").createOrReplaceTempView("t1") spark.catalog.cacheTable("t1") val query = sql("SELECT value, key from t1 group by key, value") @@ -496,9 +496,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // See PartitioningSuite for more details. withTempTable("t1", "t2") { val df1 = testData - df1.repartition(6, $"value", $"key").registerTempTable("t1") + df1.repartition(6, $"value", $"key").createOrReplaceTempView("t1") val df2 = testData2.select($"a", $"b".cast("string")) - df2.repartition(6, $"a", $"b").registerTempTable("t2") + df2.repartition(6, $"a", $"b").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index a5aecca13f..e89fa32b15 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -321,7 +321,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { nanvl($"b", $"e"), nanvl($"e", $"f")), Row(null, 3.0, 10.0, null, Double.PositiveInfinity, 3.0, 1.0) ) - testData.registerTempTable("t") + testData.createOrReplaceTempView("t") checkAnswer( sql( "select nanvl(a, 5), nanvl(b, 10), nanvl(10, b), nanvl(c, null), nanvl(d, 10), " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 4ee2006421..a15b4e1221 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -245,7 +245,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B Seq( ("2016-03-27 19:39:34", 1), ("2016-03-27 19:39:56", 2), - ("2016-03-27 19:39:27", 4)).toDF("time", "value").registerTempTable(tableName) + ("2016-03-27 19:39:27", 4)).toDF("time", "value").createOrReplaceTempView(tableName) try { f(tableName) } finally { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala index 91095af0dd..07aad3c406 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala @@ -49,7 +49,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lead") { val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( @@ -59,7 +59,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lag") { val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( @@ -70,7 +70,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lead with default value") { val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( lead("value", 2, "n/a").over(Window.partitionBy("key").orderBy("value"))), @@ -80,7 +80,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lag with default value") { val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( lag("value", 2, "n/a").over(Window.partitionBy($"key").orderBy($"value"))), @@ -89,7 +89,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("rank functions in unspecific window") { val df = Seq((1, "1"), (2, "2"), (1, "2"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( $"key", @@ -112,7 +112,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and rows between") { val df = Seq((1, "1"), (2, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( avg("key").over(Window.partitionBy($"value").orderBy($"key").rowsBetween(-1, 2))), @@ -121,7 +121,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and range between") { val df = Seq((1, "1"), (1, "1"), (3, "1"), (2, "2"), (2, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( avg("key").over(Window.partitionBy($"value").orderBy($"key").rangeBetween(-1, 1))), @@ -131,7 +131,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and rows between with unbounded") { val df = Seq((1, "1"), (2, "2"), (2, "3"), (1, "3"), (3, "2"), (4, "3")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( $"key", @@ -146,7 +146,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and range between with unbounded") { val df = Seq((5, "1"), (5, "2"), (4, "2"), (6, "2"), (3, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( $"key", @@ -357,7 +357,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and rows between with unbounded + predicate pushdown") { val df = Seq((1, "1"), (2, "2"), (2, "3"), (1, "3"), (3, "2"), (4, "3")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") val selectList = Seq($"key", $"value", last("key").over( Window.partitionBy($"value").orderBy($"key").rowsBetween(0, Long.MaxValue)), @@ -372,7 +372,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and range between with unbounded + predicate pushdown") { val df = Seq((5, "1"), (5, "2"), (4, "2"), (6, "2"), (3, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") val selectList = Seq($"key", $"value", last("value").over( Window.partitionBy($"value").orderBy($"key").rangeBetween(-2, -1)).equalTo("2") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index da567db5ee..a6b83b3d07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -344,8 +344,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { } test("full outer join") { - upperCaseData.where('N <= 4).registerTempTable("`left`") - upperCaseData.where('N >= 3).registerTempTable("`right`") + upperCaseData.where('N <= 4).createOrReplaceTempView("`left`") + upperCaseData.where('N >= 3).createOrReplaceTempView("`right`") val left = UnresolvedRelation(TableIdentifier("left"), None) val right = UnresolvedRelation(TableIdentifier("right"), None) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala index 1c6e6cc15d..65fe271b69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala @@ -29,7 +29,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex private lazy val df = (1 to 10).map(i => (i, s"str$i")).toDF("key", "value") before { - df.registerTempTable("listtablessuitetable") + df.createOrReplaceTempView("listtablessuitetable") } after { @@ -74,7 +74,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex case tableDF => assert(expectedSchema === tableDF.schema) - tableDF.registerTempTable("tables") + tableDF.createOrReplaceTempView("tables") checkAnswer( sql( "SELECT isTemporary, tableName from tables WHERE tableName = 'listtablessuitetable'"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 1d5fc570c6..38d7b6e25b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -60,7 +60,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { // temporary table should not be shared val df = session1.range(10) - df.registerTempTable("test1") + df.createOrReplaceTempView("test1") assert(session1.tableNames().contains("test1")) assert(!session2.tableNames().contains("test1")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index b67e2bdeb3..010dea5b30 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -39,7 +39,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { setupTestData() test("having clause") { - Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v").registerTempTable("hav") + Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v") + .createOrReplaceTempView("hav") checkAnswer( sql("SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2"), Row("one", 6) :: Row("three", 3) :: Nil) @@ -47,7 +48,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-8010: promote numeric to string") { val df = Seq((1, 1)).toDF("key", "value") - df.registerTempTable("src") + df.createOrReplaceTempView("src") val queryCaseWhen = sql("select case when true then 1.0 else '1' end from src ") val queryCoalesce = sql("select coalesce(null, 1, '1') from src ") @@ -100,7 +101,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { (83, 0, 38), (26, 0, 79), (43, 81, 24) - ).toDF("a", "b", "c").registerTempTable("cachedData") + ).toDF("a", "b", "c").createOrReplaceTempView("cachedData") spark.catalog.cacheTable("cachedData") checkAnswer( @@ -109,7 +110,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("self join with aliases") { - Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").registerTempTable("df") + Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").createOrReplaceTempView("df") checkAnswer( sql( @@ -137,7 +138,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { .toDF("int", "str") .groupBy("str") .agg($"str", count("str").as("strCount")) - .registerTempTable("df") + .createOrReplaceTempView("df") checkAnswer( sql( @@ -195,7 +196,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("grouping on nested fields") { spark.read.json(sparkContext.parallelize( """{"nested": {"attribute": 1}, "value": 2}""" :: Nil)) - .registerTempTable("rows") + .createOrReplaceTempView("rows") checkAnswer( sql( @@ -214,7 +215,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.read.json( sparkContext.parallelize( Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}"))) - .registerTempTable("d") + .createOrReplaceTempView("d") checkAnswer( sql("select * from d where d.a in (1,2)"), @@ -225,7 +226,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.read.json( sparkContext.parallelize( Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}", ""))) - .registerTempTable("d") + .createOrReplaceTempView("d") checkAnswer( sql("select count(1) from d"), @@ -261,7 +262,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.table("testData") .union(spark.table("testData")) .union(spark.table("testData")) - .registerTempTable("testData3x") + .createOrReplaceTempView("testData3x") try { // Just to group rows. @@ -391,7 +392,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-3173 Timestamp support in the parser") { - (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").registerTempTable("timestamps") + (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").createOrReplaceTempView("timestamps") checkAnswer(sql( "SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.0'"), @@ -746,7 +747,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("count of empty table") { withTempTable("t") { - Seq.empty[(Int, Int)].toDF("a", "b").registerTempTable("t") + Seq.empty[(Int, Int)].toDF("a", "b").createOrReplaceTempView("t") checkAnswer( sql("select count(a) from t"), Row(0)) @@ -891,10 +892,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-3349 partitioning after limit") { sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) - .registerTempTable("subset1") + .createOrReplaceTempView("subset1") sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n ASC") .limit(2) - .registerTempTable("subset2") + .createOrReplaceTempView("subset2") checkAnswer( sql("SELECT * FROM lowerCaseData INNER JOIN subset1 ON subset1.n = lowerCaseData.n"), Row(3, "c", 3) :: @@ -1111,7 +1112,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } val df1 = spark.createDataFrame(rowRDD1, schema1) - df1.registerTempTable("applySchema1") + df1.createOrReplaceTempView("applySchema1") checkAnswer( sql("SELECT * FROM applySchema1"), Row(1, "A1", true, null) :: @@ -1141,7 +1142,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } val df2 = spark.createDataFrame(rowRDD2, schema2) - df2.registerTempTable("applySchema2") + df2.createOrReplaceTempView("applySchema2") checkAnswer( sql("SELECT * FROM applySchema2"), Row(Row(1, true), Map("A1" -> null)) :: @@ -1166,7 +1167,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } val df3 = spark.createDataFrame(rowRDD3, schema2) - df3.registerTempTable("applySchema3") + df3.createOrReplaceTempView("applySchema3") checkAnswer( sql("SELECT f1.f11, f2['D4'] FROM applySchema3"), @@ -1214,7 +1215,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { def validateMetadata(rdd: DataFrame): Unit = { assert(rdd.schema("name").metadata.getString(docKey) == docValue) } - personWithMeta.registerTempTable("personWithMeta") + personWithMeta.createOrReplaceTempView("personWithMeta") validateMetadata(personWithMeta.select($"name")) validateMetadata(personWithMeta.select($"name")) validateMetadata(personWithMeta.select($"id", $"name")) @@ -1409,7 +1410,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-3483 Special chars in column names") { val data = sparkContext.parallelize( Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) - spark.read.json(data).registerTempTable("records") + spark.read.json(data).createOrReplaceTempView("records") sql("SELECT `key?number1`, `key.number2` FROM records") } @@ -1451,12 +1452,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-4322 Grouping field with struct field as sub expression") { spark.read.json(sparkContext.makeRDD("""{"a": {"b": [{"c": 1}]}}""" :: Nil)) - .registerTempTable("data") + .createOrReplaceTempView("data") checkAnswer(sql("SELECT a.b[0].c FROM data GROUP BY a.b[0].c"), Row(1)) spark.catalog.dropTempView("data") spark.read.json( - sparkContext.makeRDD("""{"a": {"b": 1}}""" :: Nil)).registerTempTable("data") + sparkContext.makeRDD("""{"a": {"b": 1}}""" :: Nil)).createOrReplaceTempView("data") checkAnswer(sql("SELECT a.b + 1 FROM data GROUP BY a.b + 1"), Row(2)) spark.catalog.dropTempView("data") } @@ -1478,10 +1479,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Supporting relational operator '<=>' in Spark SQL") { val nullCheckData1 = TestData(1, "1") :: TestData(2, null) :: Nil val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i))) - rdd1.toDF().registerTempTable("nulldata1") + rdd1.toDF().createOrReplaceTempView("nulldata1") val nullCheckData2 = TestData(1, "1") :: TestData(2, null) :: Nil val rdd2 = sparkContext.parallelize((0 to 1).map(i => nullCheckData2(i))) - rdd2.toDF().registerTempTable("nulldata2") + rdd2.toDF().createOrReplaceTempView("nulldata2") checkAnswer(sql("SELECT nulldata1.key FROM nulldata1 join " + "nulldata2 on nulldata1.value <=> nulldata2.value"), (1 to 2).map(i => Row(i))) @@ -1490,7 +1491,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Multi-column COUNT(DISTINCT ...)") { val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) - rdd.toDF().registerTempTable("distinctData") + rdd.toDF().createOrReplaceTempView("distinctData") checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2)) } @@ -1498,7 +1499,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) - rdd.toDF().registerTempTable("testTable1") + rdd.toDF().createOrReplaceTempView("testTable1") checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1")) } } @@ -1506,7 +1507,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-6145: ORDER BY test for nested fields") { spark.read.json(sparkContext.makeRDD( """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)) - .registerTempTable("nestedOrder") + .createOrReplaceTempView("nestedOrder") checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.b"), Row(1)) checkAnswer(sql("SELECT a.b FROM nestedOrder ORDER BY a.b"), Row(1)) @@ -1517,8 +1518,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-6145: special cases") { - spark.read.json(sparkContext.makeRDD( - """{"a": {"b": [1]}, "b": [{"a": 1}], "_c0": {"a": 1}}""" :: Nil)).registerTempTable("t") + spark.read + .json(sparkContext.makeRDD("""{"a": {"b": [1]}, "b": [{"a": 1}], "_c0": {"a": 1}}""" :: Nil)) + .createOrReplaceTempView("t") + checkAnswer(sql("SELECT a.b[0] FROM t ORDER BY _c0.a"), Row(1)) checkAnswer(sql("SELECT b[0].a FROM t ORDER BY _c0.a"), Row(1)) } @@ -1526,14 +1529,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-6898: complete support for special chars in column names") { spark.read.json(sparkContext.makeRDD( """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) - .registerTempTable("t") + .createOrReplaceTempView("t") checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) } test("SPARK-6583 order by aggregated function") { Seq("1" -> 3, "1" -> 4, "2" -> 7, "2" -> 8, "3" -> 5, "3" -> 6, "4" -> 1, "4" -> 2) - .toDF("a", "b").registerTempTable("orderByData") + .toDF("a", "b").createOrReplaceTempView("orderByData") checkAnswer( sql( @@ -1619,7 +1622,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { (0, null, null, false), (1, null, null, false), (null, null, null, true) - ).toDF("i", "b", "r1", "r2").registerTempTable("t") + ).toDF("i", "b", "r1", "r2").createOrReplaceTempView("t") checkAnswer(sql("select i = b from t"), sql("select r1 from t")) checkAnswer(sql("select i <=> b from t"), sql("select r2 from t")) @@ -1629,14 +1632,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-7067: order by queries for complex ExtractValue chain") { withTempTable("t") { spark.read.json(sparkContext.makeRDD( - """{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""" :: Nil)).registerTempTable("t") + """{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""" :: Nil)).createOrReplaceTempView("t") checkAnswer(sql("SELECT a.b FROM t ORDER BY b[0].d"), Row(Seq(Row(1)))) } } test("SPARK-8782: ORDER BY NULL") { withTempTable("t") { - Seq((1, 2), (1, 2)).toDF("a", "b").registerTempTable("t") + Seq((1, 2), (1, 2)).toDF("a", "b").createOrReplaceTempView("t") checkAnswer(sql("SELECT * FROM t ORDER BY NULL"), Seq(Row(1, 2), Row(1, 2))) } } @@ -1645,7 +1648,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { withTempTable("t") { val df = Seq(1 -> "a").toDF("count", "sort") checkAnswer(df.filter("count > 0"), Row(1, "a")) - df.registerTempTable("t") + df.createOrReplaceTempView("t") checkAnswer(sql("select count, sort from t"), Row(1, "a")) } } @@ -1759,7 +1762,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { withTempTable("1one") { sparkContext.parallelize(1 to 10).map(i => (i, i.toString)) .toDF("num", "str") - .registerTempTable("1one") + .createOrReplaceTempView("1one") checkAnswer(sql("select count(num) from 1one"), Row(10)) } } @@ -1801,7 +1804,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-10130 type coercion for IF should have children resolved first") { withTempTable("src") { - Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer( sql("SELECT IF(a > 0, a, 0) FROM (SELECT key a FROM src) temp"), Seq(Row(1), Row(0))) } @@ -1809,7 +1812,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-10389: order by non-attribute grouping expression on Aggregate") { withTempTable("src") { - Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1"), Seq(Row(1), Row(1))) checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY (key + 1) * 2"), @@ -1872,7 +1875,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-11032: resolve having correctly") { withTempTable("src") { - Seq(1 -> "a").toDF("i", "j").registerTempTable("src") + Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("src") checkAnswer( sql("SELECT MIN(t.i) FROM (SELECT * FROM src WHERE i > 0) t HAVING(COUNT(1) > 0)"), Row(1)) @@ -1910,8 +1913,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(1, 1, 1, 1) :: Row(1, 2, 2, 1) :: Row(2, 1, 1, 2) :: Row(2, 2, 2, 2) :: Row(3, 1, 1, 3) :: Row(3, 2, 2, 3) :: Nil) - // Try with a registered table. - sql("select struct(a, b) as record from testData2").registerTempTable("structTable") + // Try with a temporary view + sql("select struct(a, b) as record from testData2").createOrReplaceTempView("structTable") checkAnswer( sql("SELECT record.* FROM structTable"), Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) @@ -1975,9 +1978,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { nestedStructData.select($"record.r1.*"), Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) - // Try with a registered table + // Try with a temporary view withTempTable("nestedStructTable") { - nestedStructData.registerTempTable("nestedStructTable") + nestedStructData.createOrReplaceTempView("nestedStructTable") checkAnswer( sql("SELECT record.* FROM nestedStructTable"), nestedStructData.select($"record.*")) @@ -2000,7 +2003,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { | (SELECT struct(a, b) as `col$.a_`, struct(b, a) as `a.b.c.` FROM testData2) tmp """.stripMargin) withTempTable("specialCharacterTable") { - specialCharacterPath.registerTempTable("specialCharacterTable") + specialCharacterPath.createOrReplaceTempView("specialCharacterTable") checkAnswer( specialCharacterPath.select($"`r&&b.c`.*"), nestedStructData.select($"record.*")) @@ -2024,7 +2027,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { // Create a data set that contains a naming conflict val nameConflict = sql("SELECT struct(a, b) as nameConflict, a as a FROM testData2") withTempTable("nameConflict") { - nameConflict.registerTempTable("nameConflict") + nameConflict.createOrReplaceTempView("nameConflict") // Unqualified should resolve to table. checkAnswer(sql("SELECT nameConflict.* FROM nameConflict"), Row(Row(1, 1), 1) :: Row(Row(1, 2), 1) :: Row(Row(2, 1), 2) :: Row(Row(2, 2), 2) :: @@ -2328,7 +2331,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-13056: Null in map value causes NPE") { val df = Seq(1 -> Map("abc" -> "somestring", "cba" -> null)).toDF("key", "value") withTempTable("maptest") { - df.registerTempTable("maptest") + df.createOrReplaceTempView("maptest") // local optimization will by pass codegen code, so we should keep the filter `key=1` checkAnswer(sql("SELECT value['abc'] FROM maptest where key = 1"), Row("somestring")) checkAnswer(sql("SELECT value['cba'] FROM maptest where key = 1"), Row(null)) @@ -2338,7 +2341,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("hash function") { val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") withTempTable("tbl") { - df.registerTempTable("tbl") + df.createOrReplaceTempView("tbl") checkAnswer( df.select(hash($"i", $"j")), sql("SELECT hash(i, j) from tbl") @@ -2390,8 +2393,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val df1 = Seq(("one", 1), ("two", 2), ("three", 3)).toDF("k", "v1") val df2 = Seq(("one", 1), ("two", 22), ("one", 5)).toDF("k", "v2") withTempTable("nt1", "nt2") { - df1.registerTempTable("nt1") - df2.registerTempTable("nt2") + df1.createOrReplaceTempView("nt1") + df2.createOrReplaceTempView("nt2") checkAnswer( sql("SELECT * FROM nt1 natural join nt2 where k = \"one\""), Row("one", 1, 1) :: Row("one", 1, 5) :: Nil) @@ -2418,9 +2421,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val df3 = Seq((null, "r1c2", "t3r1c3"), ("r2c1", "r2c2", "t3r2c3"), ("r3c1y", "r3c2", "t3r3c3")).toDF("c1", "c2", "c3") withTempTable("t1", "t2", "t3") { - df1.registerTempTable("t1") - df2.registerTempTable("t2") - df3.registerTempTable("t3") + df1.createOrReplaceTempView("t1") + df2.createOrReplaceTempView("t2") + df3.createOrReplaceTempView("t3") // inner join with one using column checkAnswer( sql("SELECT * FROM t1 join t2 using (c1)"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 295f02f9a7..491bdb3ef9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -78,7 +78,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3)) - Seq(data).toDF().registerTempTable("reflectData") + Seq(data).toDF().createOrReplaceTempView("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === Row("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, @@ -88,7 +88,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD with nulls") { val data = NullReflectData(null, null, null, null, null, null, null) - Seq(data).toDF().registerTempTable("reflectNullData") + Seq(data).toDF().createOrReplaceTempView("reflectNullData") assert(sql("SELECT * FROM reflectNullData").collect().head === Row.fromSeq(Seq.fill(7)(null))) @@ -96,7 +96,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD with Nones") { val data = OptionalReflectData(None, None, None, None, None, None, None) - Seq(data).toDF().registerTempTable("reflectOptionalData") + Seq(data).toDF().createOrReplaceTempView("reflectOptionalData") assert(sql("SELECT * FROM reflectOptionalData").collect().head === Row.fromSeq(Seq.fill(7)(null))) @@ -104,7 +104,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { // Equality is broken for Arrays, so we test that separately. test("query binary data") { - Seq(ReflectBinary(Array[Byte](1))).toDF().registerTempTable("reflectBinary") + Seq(ReflectBinary(Array[Byte](1))).toDF().createOrReplaceTempView("reflectBinary") val result = sql("SELECT data FROM reflectBinary") .collect().head(0).asInstanceOf[Array[Byte]] @@ -124,7 +124,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), Nested(None, "abc"))) - Seq(data).toDF().registerTempTable("reflectComplexData") + Seq(data).toDF().createOrReplaceTempView("reflectComplexData") assert(sql("SELECT * FROM reflectComplexData").collect().head === Row( Seq(1, 2, 3), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 17ac0c8c6e..4819692733 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -49,9 +49,9 @@ class SubquerySuite extends QueryTest with SharedSQLContext { protected override def beforeAll(): Unit = { super.beforeAll() - l.registerTempTable("l") - r.registerTempTable("r") - t.registerTempTable("t") + l.createOrReplaceTempView("l") + r.createOrReplaceTempView("r") + t.createOrReplaceTempView("t") } test("simple uncorrelated scalar subquery") { @@ -99,7 +99,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { test("uncorrelated scalar subquery on a DataFrame generated query") { val df = Seq((1, "one"), (2, "two"), (3, "three")).toDF("key", "value") - df.registerTempTable("subqueryData") + df.createOrReplaceTempView("subqueryData") checkAnswer( sql("select (select key from subqueryData where key > 2 order by key limit 1) + 1"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 922154320c..547d3c1abe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -53,7 +53,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { test("SPARK-8003 spark_partition_id") { val df = Seq((1, "Tearing down the walls that divide us")).toDF("id", "saying") - df.registerTempTable("tmp_table") + df.createOrReplaceTempView("tmp_table") checkAnswer(sql("select spark_partition_id() from tmp_table").toDF(), Row(0)) spark.catalog.dropTempView("tmp_table") } @@ -62,7 +62,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { withTempPath { dir => val data = sparkContext.parallelize(0 to 10, 2).toDF("id") data.write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("test_table") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("test_table") val answer = sql("select input_file_name() from test_table").head().getString(0) assert(answer.contains(dir.getCanonicalPath)) assert(sql("select input_file_name() from test_table").distinct().collect().length >= 2) @@ -107,7 +107,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("integerData") + df.createOrReplaceTempView("integerData") val result = sql("SELECT * FROM integerData WHERE oneArgFilter(key)") @@ -119,7 +119,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( @@ -138,7 +138,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( @@ -158,7 +158,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 3057e016c1..7d7b486530 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -95,7 +95,7 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT test("UDTs and UDFs") { spark.udf.register("testType", (d: UDT.MyDenseVector) => d.isInstanceOf[UDT.MyDenseVector]) - pointsRDD.registerTempTable("points") + pointsRDD.createOrReplaceTempView("points") checkAnswer( sql("SELECT testType(features) from points"), Seq(Row(true), Row(true))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index d2e1ea12fd..2a5295d0d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -78,7 +78,7 @@ class PlannerSuite extends SharedSQLContext { val schema = StructType(fields) val row = Row.fromSeq(Seq.fill(fields.size)(null)) val rowRDD = sparkContext.parallelize(row :: Nil) - spark.createDataFrame(rowRDD, schema).registerTempTable("testLimit") + spark.createDataFrame(rowRDD, schema).createOrReplaceTempView("testLimit") val planned = sql( """ @@ -132,7 +132,7 @@ class PlannerSuite extends SharedSQLContext { test("InMemoryRelation statistics propagation") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "81920") { withTempTable("tiny") { - testData.limit(3).registerTempTable("tiny") + testData.limit(3).createOrReplaceTempView("tiny") sql("CACHE TABLE tiny") val a = testData.as("a") @@ -199,9 +199,9 @@ class PlannerSuite extends SharedSQLContext { test("PartitioningCollection") { withTempTable("normal", "small", "tiny") { - testData.registerTempTable("normal") - testData.limit(10).registerTempTable("small") - testData.limit(3).registerTempTable("tiny") + testData.createOrReplaceTempView("normal") + testData.limit(10).createOrReplaceTempView("small") + testData.limit(3).createOrReplaceTempView("tiny") // Disable broadcast join withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index b31338e827..bf3a39c84b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -134,7 +134,8 @@ class AggregateBenchmark extends BenchmarkBase { val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) - sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k").registerTempTable("test") + sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") + .createOrReplaceTempView("test") def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 2099d4e1b3..e2fb91352d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -42,7 +42,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { test("default size avoids broadcast") { // TODO: Improve this test when we have better statistics sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)) - .toDF().registerTempTable("sizeTst") + .toDF().createOrReplaceTempView("sizeTst") spark.catalog.cacheTable("sizeTst") assert( spark.table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes > @@ -92,7 +92,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-2729 regression: timestamp data type") { val timestamps = (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time") - timestamps.registerTempTable("timestamps") + timestamps.createOrReplaceTempView("timestamps") checkAnswer( sql("SELECT time FROM timestamps"), @@ -133,7 +133,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { assert(df.schema.head.dataType === DecimalType(15, 10)) - df.cache().registerTempTable("test_fixed_decimal") + df.cache().createOrReplaceTempView("test_fixed_decimal") checkAnswer( sql("SELECT * FROM test_fixed_decimal"), (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) @@ -179,7 +179,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { (i to i + 10).map(j => s"map_key_$j" -> (Long.MaxValue - j)).toMap, Row((i - 0.25).toFloat, Seq(true, false, null))) } - spark.createDataFrame(rdd, schema).registerTempTable("InMemoryCache_different_data_types") + spark.createDataFrame(rdd, schema).createOrReplaceTempView("InMemoryCache_different_data_types") // Cache the table. sql("cache table InMemoryCache_different_data_types") // Make sure the table is indeed cached. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index 48c798986b..a118cec0bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -63,7 +63,7 @@ class PartitionBatchPruningSuite val string = if (((key - 1) / 10) % 2 == 0) null else key.toString TestData(key, string) }, 5).toDF() - pruningData.registerTempTable("pruningData") + pruningData.createOrReplaceTempView("pruningData") spark.catalog.cacheTable("pruningData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 63fe4658d6..46213a22ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -239,7 +239,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructField("nullstr", StringType, true):: Nil) assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select nullstr, headers.Host from jsonTable"), @@ -261,7 +261,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -302,7 +302,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -376,7 +376,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("GetField operation on complex data type") { val jsonDF = spark.read.json(complexFieldAndType1) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), @@ -403,7 +403,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -464,7 +464,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ignore("Type conflict in primitive field values (Ignored)") { val jsonDF = spark.read.json(primitiveFieldValueTypeConflict) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Right now, the analyzer does not promote strings in a boolean expression. // Number and Boolean conflict: resolve the type as boolean in this query. @@ -528,7 +528,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -550,7 +550,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -580,7 +580,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") } test("Loading a JSON dataset from a text file") { @@ -601,7 +601,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -633,7 +633,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -674,7 +674,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -759,7 +759,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -885,7 +885,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(schema === jsonDF1.schema) - jsonDF1.registerTempTable("jsonTable1") + jsonDF1.createOrReplaceTempView("jsonTable1") checkAnswer( sql("select * from jsonTable1"), @@ -902,7 +902,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(schema === jsonDF2.schema) - jsonDF2.registerTempTable("jsonTable2") + jsonDF2.createOrReplaceTempView("jsonTable2") checkAnswer( sql("select * from jsonTable2"), @@ -921,7 +921,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1) - jsonWithSimpleMap.registerTempTable("jsonWithSimpleMap") + jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap") checkAnswer( sql("select `map` from jsonWithSimpleMap"), @@ -949,7 +949,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2) - jsonWithComplexMap.registerTempTable("jsonWithComplexMap") + jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap") checkAnswer( sql("select `map` from jsonWithComplexMap"), @@ -974,7 +974,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-2096 Correctly parse dot notations") { val jsonDF = spark.read.json(complexFieldAndType2) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), @@ -992,7 +992,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-3390 Complex arrays") { val jsonDF = spark.read.json(complexFieldAndType2) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql( @@ -1015,7 +1015,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-3308 Read top level JSON arrays") { val jsonDF = spark.read.json(jsonArray) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql( @@ -1084,7 +1084,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { withTempTable("jsonTable") { val jsonDF = spark.read.json(corruptRecords) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") val schema = StructType( StructField("_unparsed", StringType, true) :: StructField("a", StringType, true) :: @@ -1156,7 +1156,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-4068: nulls in arrays") { val jsonDF = spark.read.json(nullsInArrays) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") val schema = StructType( StructField("field1", @@ -1202,7 +1202,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } val df1 = spark.createDataFrame(rowRDD1, schema1) - df1.registerTempTable("applySchema1") + df1.createOrReplaceTempView("applySchema1") val df2 = df1.toDF val result = df2.toJSON.collect() // scalastyle:off @@ -1225,7 +1225,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } val df3 = spark.createDataFrame(rowRDD2, schema2) - df3.registerTempTable("applySchema2") + df3.createOrReplaceTempView("applySchema2") val df4 = df3.toDF val result2 = df4.toJSON.collect() @@ -1234,7 +1234,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val jsonDF = spark.read.json(primitiveFieldAndType) val primTable = spark.read.json(jsonDF.toJSON.rdd) - primTable.registerTempTable("primitiveTable") + primTable.createOrReplaceTempView("primitiveTable") checkAnswer( sql("select * from primitiveTable"), Row(new java.math.BigDecimal("92233720368547758070"), @@ -1247,7 +1247,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val complexJsonDF = spark.read.json(complexFieldAndType1) val compTable = spark.read.json(complexJsonDF.toJSON.rdd) - compTable.registerTempTable("complexTable") + compTable.createOrReplaceTempView("complexTable") // Access elements of a primitive array. checkAnswer( sql("select arrayOfString[0], arrayOfString[1], arrayOfString[2] from complexTable"), @@ -1387,7 +1387,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { "col1", "abd") - spark.read.json(root.getAbsolutePath).registerTempTable("test_myjson_with_part") + spark.read.json(root.getAbsolutePath).createOrReplaceTempView("test_myjson_with_part") checkAnswer(sql( "SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abc'"), Row(4)) checkAnswer(sql( @@ -1531,7 +1531,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { { val jsonDF = spark.read.schema(schema).json(additionalCorruptRecords) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // In HiveContext, backticks should be used to access columns starting with a underscore. checkAnswer( @@ -1639,7 +1639,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val schema = (new StructType).add("ts", TimestampType) val jsonDF = spark.read.schema(schema).json(timestampAsLong) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select ts from jsonTable"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 8707e13461..847ea6bd52 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -400,7 +400,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha // Introduce _temporary dir to the base dir the robustness of the schema discovery process. new File(base.getCanonicalPath, "_temporary").mkdir() - spark.read.parquet(base.getCanonicalPath).registerTempTable("t") + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -484,7 +484,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - spark.read.parquet(base.getCanonicalPath).registerTempTable("t") + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -533,7 +533,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) - parquetRelation.registerTempTable("t") + parquetRelation.createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -573,7 +573,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) - parquetRelation.registerTempTable("t") + parquetRelation.createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -609,7 +609,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha .option("mergeSchema", "true") .format("parquet") .load(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") withTempTable("t") { checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index f9f9f80352..725e14c0fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -46,7 +46,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("appending") { val data = (0 until 10).map(i => (i, i.toString)) - spark.createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") // Query appends, don't test with both read modes. withParquetTable(data, "t", false) { sql("INSERT INTO TABLE t SELECT * FROM tmp") @@ -58,7 +58,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("overwriting") { val data = (0 until 10).map(i => (i, i.toString)) - spark.createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withParquetTable(data, "t") { sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") checkAnswer(spark.table("t"), data.map(Row.fromTuple)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala index 69a600a55b..487d7a7e5a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala @@ -75,10 +75,10 @@ object ParquetReadBenchmark { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select cast(id as INT) as id from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") sqlBenchmark.addCase("SQL Parquet Vectorized") { iter => spark.sql("select sum(id) from tempTable").collect() @@ -159,10 +159,10 @@ object ParquetReadBenchmark { def intStringScanBenchmark(values: Int): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select cast(id as INT) as c1, cast(id as STRING) as c2 from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("Int and String Scan", values) @@ -193,10 +193,10 @@ object ParquetReadBenchmark { def stringDictionaryScanBenchmark(values: Int): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select cast((id % 200) + 10000 as STRING) as c1 from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("String Dictionary", values) @@ -225,10 +225,10 @@ object ParquetReadBenchmark { def partitionTableScanBenchmark(values: Int): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select id % 2 as p, cast(id as INT) as id from t1") .write.partitionBy("p").parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("Partitioned Table", values) @@ -260,11 +260,11 @@ object ParquetReadBenchmark { def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql(s"select IF(rand(1) < $fractionOfNulls, NULL, cast(id as STRING)) as c1, " + s"IF(rand(2) < $fractionOfNulls, NULL, cast(id as STRING)) as c2 from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("String with Nulls Scan", values) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala index 08b7eb3cf7..228ae6f840 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala @@ -1187,7 +1187,7 @@ object TPCDSBenchmark { def setupTables(dataLocation: String): Map[String, Long] = { tables.map { tableName => - spark.read.parquet(s"$dataLocation/$tableName").registerTempTable(tableName) + spark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName) tableName -> spark.table(tableName).count() }.toMap } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 1b82769428..08f596f130 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -165,7 +165,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { // Because SortMergeJoin may skip different rows if the number of partitions is different, this // test should use the deterministic number of partitions. val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) @@ -183,7 +183,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { // Because SortMergeJoin may skip different rows if the number of partitions is different, // this test should use the deterministic number of partitions. val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) @@ -237,7 +237,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("BroadcastNestedLoopJoin metrics") { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> BroadcastNestedLoopJoin(nodeId = 1) -> TungstenProject(nodeId = 0) @@ -265,7 +265,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("CartesianProduct metrics") { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 44d1b9ddda..9c9abfeb2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -441,7 +441,7 @@ class JDBCSuite extends SparkFunSuite test("test DATE types in cache") { val rows = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties).collect() spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties) - .cache().registerTempTable("mycached_date") + .cache().createOrReplaceTempView("mycached_date") val cachedRows = sql("select * from mycached_date").collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index c1dc9b9834..03c18ad009 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -34,7 +34,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with super.beforeAll() path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) - caseInsensitiveContext.read.json(rdd).registerTempTable("jt") + caseInsensitiveContext.read.json(rdd).createOrReplaceTempView("jt") } override def afterAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 5ac39f54b9..854fec5b22 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -31,7 +31,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { super.beforeAll() path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - caseInsensitiveContext.read.json(rdd).registerTempTable("jt") + caseInsensitiveContext.read.json(rdd).createOrReplaceTempView("jt") sql( s""" |CREATE TEMPORARY TABLE jsonTable (a int, b string) @@ -111,7 +111,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to less part files. val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 5) - caseInsensitiveContext.read.json(rdd1).registerTempTable("jt1") + caseInsensitiveContext.read.json(rdd1).createOrReplaceTempView("jt1") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt1 @@ -123,7 +123,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to more part files. val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 10) - caseInsensitiveContext.read.json(rdd2).registerTempTable("jt2") + caseInsensitiveContext.read.json(rdd2).createOrReplaceTempView("jt2") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index bb2c54aa64..7738e4107d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -42,7 +42,7 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) df = caseInsensitiveContext.read.json(rdd) - df.registerTempTable("jsonTable") + df.createOrReplaceTempView("jsonTable") } override def afterAll(): Unit = { @@ -123,7 +123,7 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA // verify the append mode df.write.mode(SaveMode.Append).json(path.toString) val df2 = df.union(df) - df2.registerTempTable("jsonTable2") + df2.createOrReplaceTempView("jsonTable2") checkLoad(df2, "jsonTable2") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 013b731693..b742206b58 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -77,7 +77,7 @@ class StreamSuite extends StreamTest with SharedSQLContext { test("sql queries") { val inputData = MemoryStream[Int] - inputData.toDF().registerTempTable("stream") + inputData.toDF().createOrReplaceTempView("stream") val evens = sql("SELECT * FROM stream WHERE value % 2 = 0") testStream(evens)( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 03369c5a48..421f6bca7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -41,14 +41,14 @@ private[sql] trait SQLTestData { self => protected lazy val emptyTestData: DataFrame = { val df = spark.sparkContext.parallelize( Seq.empty[Int].map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("emptyTestData") + df.createOrReplaceTempView("emptyTestData") df } protected lazy val testData: DataFrame = { val df = spark.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("testData") + df.createOrReplaceTempView("testData") df } @@ -60,7 +60,7 @@ private[sql] trait SQLTestData { self => TestData2(2, 2) :: TestData2(3, 1) :: TestData2(3, 2) :: Nil, 2).toDF() - df.registerTempTable("testData2") + df.createOrReplaceTempView("testData2") df } @@ -68,14 +68,14 @@ private[sql] trait SQLTestData { self => val df = spark.sparkContext.parallelize( TestData3(1, None) :: TestData3(2, Some(2)) :: Nil).toDF() - df.registerTempTable("testData3") + df.createOrReplaceTempView("testData3") df } protected lazy val negativeData: DataFrame = { val df = spark.sparkContext.parallelize( (1 to 100).map(i => TestData(-i, (-i).toString))).toDF() - df.registerTempTable("negativeData") + df.createOrReplaceTempView("negativeData") df } @@ -87,7 +87,7 @@ private[sql] trait SQLTestData { self => LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: LargeAndSmallInts(3, 2) :: Nil).toDF() - df.registerTempTable("largeAndSmallInts") + df.createOrReplaceTempView("largeAndSmallInts") df } @@ -99,7 +99,7 @@ private[sql] trait SQLTestData { self => DecimalData(2, 2) :: DecimalData(3, 1) :: DecimalData(3, 2) :: Nil).toDF() - df.registerTempTable("decimalData") + df.createOrReplaceTempView("decimalData") df } @@ -110,7 +110,7 @@ private[sql] trait SQLTestData { self => BinaryData("122".getBytes(StandardCharsets.UTF_8), 3) :: BinaryData("121".getBytes(StandardCharsets.UTF_8), 2) :: BinaryData("123".getBytes(StandardCharsets.UTF_8), 4) :: Nil).toDF() - df.registerTempTable("binaryData") + df.createOrReplaceTempView("binaryData") df } @@ -122,7 +122,7 @@ private[sql] trait SQLTestData { self => UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: UpperCaseData(6, "F") :: Nil).toDF() - df.registerTempTable("upperCaseData") + df.createOrReplaceTempView("upperCaseData") df } @@ -132,7 +132,7 @@ private[sql] trait SQLTestData { self => LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: LowerCaseData(4, "d") :: Nil).toDF() - df.registerTempTable("lowerCaseData") + df.createOrReplaceTempView("lowerCaseData") df } @@ -140,7 +140,7 @@ private[sql] trait SQLTestData { self => val rdd = spark.sparkContext.parallelize( ArrayData(Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: ArrayData(Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) - rdd.toDF().registerTempTable("arrayData") + rdd.toDF().createOrReplaceTempView("arrayData") rdd } @@ -151,13 +151,13 @@ private[sql] trait SQLTestData { self => MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: MapData(Map(1 -> "a4", 2 -> "b4")) :: MapData(Map(1 -> "a5")) :: Nil) - rdd.toDF().registerTempTable("mapData") + rdd.toDF().createOrReplaceTempView("mapData") rdd } protected lazy val repeatedData: RDD[StringData] = { val rdd = spark.sparkContext.parallelize(List.fill(2)(StringData("test"))) - rdd.toDF().registerTempTable("repeatedData") + rdd.toDF().createOrReplaceTempView("repeatedData") rdd } @@ -165,7 +165,7 @@ private[sql] trait SQLTestData { self => val rdd = spark.sparkContext.parallelize( List.fill(2)(StringData(null)) ++ List.fill(2)(StringData("test"))) - rdd.toDF().registerTempTable("nullableRepeatedData") + rdd.toDF().createOrReplaceTempView("nullableRepeatedData") rdd } @@ -175,7 +175,7 @@ private[sql] trait SQLTestData { self => NullInts(2) :: NullInts(3) :: NullInts(null) :: Nil).toDF() - df.registerTempTable("nullInts") + df.createOrReplaceTempView("nullInts") df } @@ -185,7 +185,7 @@ private[sql] trait SQLTestData { self => NullInts(null) :: NullInts(null) :: NullInts(null) :: Nil).toDF() - df.registerTempTable("allNulls") + df.createOrReplaceTempView("allNulls") df } @@ -194,13 +194,13 @@ private[sql] trait SQLTestData { self => NullStrings(1, "abc") :: NullStrings(2, "ABC") :: NullStrings(3, null) :: Nil).toDF() - df.registerTempTable("nullStrings") + df.createOrReplaceTempView("nullStrings") df } protected lazy val tableName: DataFrame = { val df = spark.sparkContext.parallelize(TableName("test") :: Nil).toDF() - df.registerTempTable("tableName") + df.createOrReplaceTempView("tableName") df } @@ -215,7 +215,7 @@ private[sql] trait SQLTestData { self => // An RDD with 4 elements and 8 partitions protected lazy val withEmptyParts: RDD[IntField] = { val rdd = spark.sparkContext.parallelize((1 to 4).map(IntField), 8) - rdd.toDF().registerTempTable("withEmptyParts") + rdd.toDF().createOrReplaceTempView("withEmptyParts") rdd } @@ -223,7 +223,7 @@ private[sql] trait SQLTestData { self => val df = spark.sparkContext.parallelize( Person(0, "mike", 30) :: Person(1, "jim", 20) :: Nil).toDF() - df.registerTempTable("person") + df.createOrReplaceTempView("person") df } @@ -231,7 +231,7 @@ private[sql] trait SQLTestData { self => val df = spark.sparkContext.parallelize( Salary(0, 2000.0) :: Salary(1, 1000.0) :: Nil).toDF() - df.registerTempTable("salary") + df.createOrReplaceTempView("salary") df } @@ -240,7 +240,7 @@ private[sql] trait SQLTestData { self => ComplexData(Map("1" -> 1), TestData(1, "1"), Seq(1, 1, 1), true) :: ComplexData(Map("2" -> 2), TestData(2, "2"), Seq(2, 2, 2), false) :: Nil).toDF() - df.registerTempTable("complexData") + df.createOrReplaceTempView("complexData") df } @@ -251,7 +251,7 @@ private[sql] trait SQLTestData { self => CourseSales("dotNET", 2012, 5000) :: CourseSales("dotNET", 2013, 48000) :: CourseSales("Java", 2013, 30000) :: Nil).toDF() - df.registerTempTable("courseSales") + df.createOrReplaceTempView("courseSales") df } |