From eda83158f603139d9ef509376c3463ed14c53c3e Mon Sep 17 00:00:00 2001 From: rbenrejeb Date: Sun, 18 Aug 2024 16:18:21 +0200 Subject: [PATCH] PHOENIX-6783 Add spark-sql for spark2 and spark3 --- phoenix5-spark/README.md | 145 +++- .../apache/phoenix/spark/DataSourceApiIT.java | 7 +- .../org/apache/phoenix/spark/OrderByIT.java | 24 +- .../apache/phoenix/spark/SparkResultSet.java | 0 .../org/apache/phoenix/spark/SparkUtil.java | 4 +- .../src/it/resources/globalSetup.sql | 5 +- .../src/it/resources/tenantSetup.sql | 2 + .../spark/AbstractPhoenixSparkIT.scala | 32 +- .../spark/PhoenixSparkDatasourceV1IT.scala | 743 ++++++++++++++++++ ...nixSparkDatasourceV1ITTenantSpecific.scala | 117 +++ .../apache/phoenix/spark/PhoenixSparkIT.scala | 170 ++-- .../spark/PhoenixSparkITTenantSpecific.scala | 167 ++-- .../phoenix/spark/PhoenixSparkSqlIT.scala | 113 +++ .../datasource/v2/PhoenixDataSource.java | 40 +- .../v2/reader/PhoenixDataSourceReader.java | 54 +- .../reader/PhoenixInputPartitionReader.java | 1 - .../v2/writer/PhoenixDataSourceWriter.java | 3 +- .../spark/FilterExpressionCompiler.scala | 12 +- .../org/apache/phoenix/spark/PhoenixRDD.scala | 1 - .../phoenix/spark/PhoenixRelation.scala | 2 +- .../phoenix/spark/SparkSchemaUtil.scala | 13 +- .../v2/PhoenixSparkSqlRelation.scala | 58 ++ phoenix5-spark3/README.md | 137 +++- .../apache/phoenix/spark/DataSourceApiIT.java | 21 +- .../org/apache/phoenix/spark/OrderByIT.java | 36 +- .../apache/phoenix/spark/SparkResultSet.java | 0 .../org/apache/phoenix/spark/SparkUtil.java | 7 +- .../connector/PhoenixTestingDataSource.java | 7 +- .../src/it/resources/globalSetup.sql | 10 +- .../src/it/resources/tenantSetup.sql | 2 + .../spark/AbstractPhoenixSparkIT.scala | 45 +- .../spark/PhoenixSparkDatasourceV1IT.scala | 737 +++++++++++++++++ ...nixSparkDatasourceV1ITTenantSpecific.scala | 117 +++ .../apache/phoenix/spark/PhoenixSparkIT.scala | 149 ++-- .../spark/PhoenixSparkITTenantSpecific.scala | 194 +++-- .../phoenix/spark/PhoenixSparkSqlIT.scala | 113 +++ .../sql/connector/PhoenixDataSource.java | 76 +- .../spark/sql/connector/PhoenixTable.java | 10 +- .../sql/connector/reader/PhoenixScan.java | 4 +- .../connector/reader/PhoenixScanBuilder.java | 3 +- .../connector/writer/PhoenixBatchWrite.java | 5 +- .../phoenix/spark/ConfigurationUtil.scala | 3 +- .../spark/FilterExpressionCompiler.scala | 12 +- .../org/apache/phoenix/spark/PhoenixRDD.scala | 1 - .../phoenix/spark/PhoenixRelation.scala | 2 +- .../phoenix/spark/SparkSchemaUtil.scala | 15 +- .../connector/PhoenixSparkSqlRelation.scala | 58 ++ .../datasources/jdbc/SparkJdbcUtil.scala | 1 - 48 files changed, 2865 insertions(+), 613 deletions(-) rename phoenix5-spark/src/{main => it}/java/org/apache/phoenix/spark/SparkResultSet.java (100%) create mode 100644 phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala create mode 100644 phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala create mode 100644 phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala create mode 100644 phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/PhoenixSparkSqlRelation.scala rename phoenix5-spark3/src/{main => it}/java/org/apache/phoenix/spark/SparkResultSet.java (100%) create mode 100644 phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala create mode 100644 phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala create mode 100644 phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala create mode 100644 phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/PhoenixSparkSqlRelation.scala diff --git a/phoenix5-spark/README.md b/phoenix5-spark/README.md index 3542a19d..0d2ebcca 100644 --- a/phoenix5-spark/README.md +++ b/phoenix5-spark/README.md @@ -18,6 +18,19 @@ limitations under the License. phoenix-spark extends Phoenix's MapReduce support to allow Spark to load Phoenix tables as DataFrames, and enables persisting DataFrames back to Phoenix. +## Configuration properties + +| Name | Default | Usage | Description | +|---------------------------|---------|-------|-----------------------------------------------------------------------------------------------------------------------------------------------| +| table | empty | R/W | table name as `namespace.table_name` | +| zkUrl | empty | R/W | (Optional) List of zookeeper hosts. Deprecated, use `jdbcUrl` instead. Recommended not to set, value will be taken from hbase-site.xml | +| jdbcUrl | empty | R/W | (Optional) jdbc url connection to database as `jdbc:phoenix:zkHost:zkport`. Recommended not to set, value will be taken from hbase-site.xml | +| dateAsTimestamp | false | R | Cast Date to Timestamp | +| doNotMapColumnFamily | false | R | For non default column family. Do not prefix column with column family name | +| TenantId | empty | R/W | Define tenantId when reading from multitenant table | +| phoenixconfigs | empty | R/W | Comma seperated value of hbase/phoenix config to override. (property=value,property=value) | +| skipNormalizingIdentifier | empty | W | skip normalize identifier | + ## Reading Phoenix Tables Given a Phoenix table with the following DDL and DML: @@ -45,12 +58,14 @@ val spark = SparkSession val df = spark.sqlContext .read .format("phoenix") - .options(Map("table" -> "TABLE1")) + .option("table", "TABLE1") .load df.filter(df("COL1") === "test_row_1" && df("ID") === 1L) .select(df("ID")) .show + +spark.stop() ``` Java example: ```java @@ -65,21 +80,68 @@ public class PhoenixSparkRead { public static void main() throws Exception { SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test") .set("spark.hadoopRDD.ignoreEmptySplits", "false"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSessinon spark = SparkSession.builder().config(sparkConf).getOrCreate(); // Load data from TABLE1 - Dataset df = sqlContext + Dataset df = spark .read() .format("phoenix") .option("table", "TABLE1") .load(); df.createOrReplaceTempView("TABLE1"); - SQLContext sqlCtx = new SQLContext(jsc); - df = sqlCtx.sql("SELECT * FROM TABLE1 WHERE COL1='test_row_1' AND ID=1L"); + df = spark.sql("SELECT * FROM TABLE1 WHERE COL1='test_row_1' AND ID=1L"); df.show(); - jsc.stop(); + + spark.stop(); + } +} +``` + +### Load as a DataFrame using SparkSql and the DataSourceV2 API +Scala example: +```scala +import org.apache.spark.SparkContext +import org.apache.spark.sql.{SQLContext, SparkSession} + +val spark = SparkSession + .builder() + .appName("phoenix-test") + .master("local") + .config("spark.hadoopRDD.ignoreEmptySplits", "false") + .getOrCreate() + +// Load data from TABLE1 +spark.sql("CREATE TABLE TABLE1_SQL USING phoenix OPTIONS ('table' 'TABLE1')") + +val df = spark.sql(s"SELECT ID FROM TABLE1_SQL where COL1='test_row_1'") + +df.show + +spark.stop() +``` +Java example: +```java +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; + +public class PhoenixSparkRead { + + public static void main() throws Exception { + SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test") + .set("spark.hadoopRDD.ignoreEmptySplits", "false"); + SparkSessinon spark = SparkSession.builder().config(sparkConf).getOrCreate(); + + // Load data from TABLE1 + Dataset df = spark.sql("CREATE TABLE TABLE1_SQL USING phoenix OPTIONS ('table' 'TABLE1'"); + + df = spark.sql("SELECT * FROM TABLE1_SQL WHERE COL1='test_row_1' AND ID=1L"); + df.show(); + + spark.stop(); } } ``` @@ -89,8 +151,8 @@ public class PhoenixSparkRead { ### Save DataFrames to Phoenix using DataSourceV2 The `save` is method on DataFrame allows passing in a data source type. You can use -`phoenix` for DataSourceV2 and must also pass in a `table` and `zkUrl` parameter to -specify which table and server to persist the DataFrame to. The column names are derived from +`phoenix` for DataSourceV2 and must also pass in a `table` parameter to +specify which table to persist the DataFrame to. The column names are derived from the DataFrame's schema field names, and must match the Phoenix column names. The `save` method also takes a `SaveMode` option, for which only `SaveMode.Overwrite` is supported. @@ -118,15 +180,16 @@ val spark = SparkSession val df = spark.sqlContext .read .format("phoenix") - .options(Map("table" -> "INPUT_TABLE")) + .option("table", "INPUT_TABLE") .load // Save to OUTPUT_TABLE df.write .format("phoenix") .mode(SaveMode.Overwrite) - .options(Map("table" -> "OUTPUT_TABLE")) + .option("table", "OUTPUT_TABLE") .save() +spark.stop() ``` Java example: ```java @@ -141,12 +204,11 @@ public class PhoenixSparkWriteFromInputTable { public static void main() throws Exception { SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test") - .set("spark.hadoopRDD.ignoreEmptySplits", "false"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(jsc); + .set("spark.hadoopRDD.ignoreEmptySplits", "false"); + SparkSessinon spark = SparkSession.builder().config(sparkConf).getOrCreate(); // Load INPUT_TABLE - Dataset df = sqlContext + Dataset df = spark .read() .format("phoenix") .option("table", "INPUT_TABLE") @@ -158,15 +220,16 @@ public class PhoenixSparkWriteFromInputTable { .mode(SaveMode.Overwrite) .option("table", "OUTPUT_TABLE") .save(); - jsc.stop(); + + spark.stop(); } } ``` ### Save from an external RDD with a schema to a Phoenix table -Just like the previous example, you can pass in the data source type as `phoenix` and specify the `table` and -`zkUrl` parameters indicating which table and server to persist the DataFrame to. +Just like the previous example, you can pass in the data source type as `phoenix` and specify the `table` parameter +indicating which table to persist the DataFrame to. Note that the schema of the RDD must match its column data and this must match the schema of the Phoenix table that you save to. @@ -200,13 +263,15 @@ val schema = StructType( val rowRDD = spark.sparkContext.parallelize(dataSet) // Apply the schema to the RDD. -val df = spark.sqlContext.createDataFrame(rowRDD, schema) +val df = spark.createDataFrame(rowRDD, schema) df.write .format("phoenix") - .options(Map("table" -> "OUTPUT_TABLE")) + .option("table", "OUTPUT_TABLE") .mode(SaveMode.Overwrite) .save() + +spark.stop() ``` Java example: ```java @@ -230,10 +295,7 @@ public class PhoenixSparkWriteFromRDDWithSchema { public static void main() throws Exception { SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test") .set("spark.hadoopRDD.ignoreEmptySplits", "false"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(jsc); - SparkSession spark = sqlContext.sparkSession(); - Dataset df; + SparkSessinon spark = SparkSession.builder().config(sparkConf).getOrCreate(); // Generate the schema based on the fields List fields = new ArrayList<>(); @@ -249,14 +311,14 @@ public class PhoenixSparkWriteFromRDDWithSchema { } // Create a DataFrame from the rows and the specified schema - df = spark.createDataFrame(rows, schema); + Dataset df = spark.createDataFrame(rows, schema); df.write() .format("phoenix") .mode(SaveMode.Overwrite) .option("table", "OUTPUT_TABLE") .save(); - - jsc.stop(); + + spark.stop(); } } ``` @@ -270,13 +332,13 @@ it falls back to using connection defined by hbase-site.xml. - `"jdbcUrl"` expects a full Phoenix JDBC URL, i.e. "jdbc:phoenix" or "jdbc:phoenix:zkHost:zkport", while `"zkUrl"` expects the ZK quorum only, i.e. "zkHost:zkPort" - If you want to use DataSourceV1, you can use source type `"org.apache.phoenix.spark"` -instead of `"phoenix"`, however this is deprecated as of `connectors-1.0.0`. -The `"org.apache.phoenix.spark"` datasource does not accept the `"jdbcUrl"` parameter, -only `"zkUrl"` + instead of `"phoenix"`, however this is deprecated. + The `"org.apache.phoenix.spark"` datasource does not accept the `"jdbcUrl"` parameter, + only `"zkUrl"` - The (deprecated) functions `phoenixTableAsDataFrame`, `phoenixTableAsRDD` and -`saveToPhoenix` use the deprecated `"org.apache.phoenix.spark"` datasource, and allow -optionally specifying a `conf` Hadoop configuration parameter with custom Phoenix client settings, -as well as an optional `zkUrl` parameter. + `saveToPhoenix` use the deprecated `"org.apache.phoenix.spark"` datasource, and allow + optionally specifying a `conf` Hadoop configuration parameter with custom Phoenix client settings, + as well as an optional `zkUrl` parameter. - As of [PHOENIX-5197](https://issues.apache.org/jira/browse/PHOENIX-5197), you can pass configurations from the driver to executors as a comma-separated list against the key `phoenixConfigs` i.e (PhoenixDataSource.PHOENIX_CONFIGS), for ex: @@ -285,7 +347,7 @@ to executors as a comma-separated list against the key `phoenixConfigs` i.e (Pho .sqlContext .read .format("phoenix") - .options(Map("table" -> "Table1", "jdbcUrl" -> "jdbc:phoenix:phoenix-server:2181", "phoenixConfigs" -> "hbase.client.retries.number=10,hbase.client.pause=10000")) + .options(Map("table" -> "Table1", "phoenixConfigs" -> "hbase.client.retries.number=10,hbase.client.pause=10000")) .load; ``` This list of properties is parsed and populated into a properties map which is passed to @@ -300,7 +362,7 @@ to executors as a comma-separated list against the key `phoenixConfigs` i.e (Pho .sqlContext .read .format("phoenix") - .options(Map("table" -> "Table1", "jdbcUrl" -> "jdbc:phoenix:phoenix-server:2181", "doNotMapColumnFamily" -> "true")) + .options(Map("table" -> "Table1", "doNotMapColumnFamily" -> "true")) .load; ``` @@ -316,6 +378,7 @@ create the DataFrame or RDD directly if you need fine-grained configuration. ### Load as a DataFrame directly using a Configuration object ```scala import org.apache.hadoop.conf.Configuration +import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.sql.SQLContext import org.apache.phoenix.spark._ @@ -323,7 +386,7 @@ import org.apache.phoenix.spark._ val configuration = new Configuration() // Can set Phoenix-specific settings, requires 'hbase.zookeeper.quorum' -val sparkConf = new SparkConf().set("spark.ui.showConsoleProgress", "false") +val sparkConf = new SparkConf() val sc = new SparkContext("local", "phoenix-test", sparkConf) val sqlContext = new SQLContext(sc) @@ -337,13 +400,13 @@ df.show ### Load as an RDD, using a Zookeeper URL ```scala +import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.sql.SQLContext import org.apache.phoenix.spark._ import org.apache.spark.rdd.RDD -val sparkConf = new SparkConf().set("spark.ui.showConsoleProgress", "false") -val sc = new SparkContext("local", "phoenix-test", sparkConf) +val sc = new SparkContext("local", "phoenix-test") // Load the columns 'ID' and 'COL1' from TABLE1 as an RDD val rdd: RDD[Map[String, AnyRef]] = sc.phoenixTableAsRDD( @@ -368,10 +431,12 @@ CREATE TABLE OUTPUT_TEST_TABLE (id BIGINT NOT NULL PRIMARY KEY, col1 VARCHAR, co ``` ```scala +import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.phoenix.spark._ -val sc = new SparkContext("local", "phoenix-test") +val sparkConf = new SparkConf() +val sc = new SparkContext("local", "phoenix-test", sparkConf) val dataSet = List((1L, "1", 1), (2L, "2", 2), (3L, "3", 3)) sc @@ -381,4 +446,4 @@ sc Seq("ID","COL1","COL2"), zkUrl = Some("phoenix-server:2181") ) -``` \ No newline at end of file +``` diff --git a/phoenix5-spark/src/it/java/org/apache/phoenix/spark/DataSourceApiIT.java b/phoenix5-spark/src/it/java/org/apache/phoenix/spark/DataSourceApiIT.java index 507fc8e4..28f5f621 100644 --- a/phoenix5-spark/src/it/java/org/apache/phoenix/spark/DataSourceApiIT.java +++ b/phoenix5-spark/src/it/java/org/apache/phoenix/spark/DataSourceApiIT.java @@ -31,7 +31,6 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -148,7 +147,7 @@ public void basicWriteAndReadBackTest() throws SQLException { Dataset df1Read = spark.read().format("phoenix") .option("table", tableName) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); assertEquals(3l, df1Read.count()); @@ -173,7 +172,6 @@ public void basicWriteAndReadBackTest() throws SQLException { } @Test - @Ignore // Spark3 seems to be unable to handle mixed case colum names public void lowerCaseWriteTest() throws SQLException { SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test") .set("spark.hadoopRDD.ignoreEmptySplits", "false"); @@ -205,7 +203,8 @@ public void lowerCaseWriteTest() throws SQLException { .format("phoenix") .mode(SaveMode.Overwrite) .option("table", tableName) - .option(ZOOKEEPER_URL, getUrl()) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER,"true") + .option(JDBC_URL, getUrl()) .save(); try (Connection conn = DriverManager.getConnection(getUrl()); diff --git a/phoenix5-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java b/phoenix5-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java index 342edae0..6019d589 100644 --- a/phoenix5-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java +++ b/phoenix5-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java @@ -48,11 +48,6 @@ import org.junit.experimental.categories.Category; import org.junit.Ignore; - - -import scala.Option; -import scala.collection.JavaConverters; - @Category(ParallelStatsDisabledTest.class) public class OrderByIT extends BaseOrderByIT { @@ -135,11 +130,11 @@ public void testOrderByWithJoin() throws Exception { SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") .option(DataSourceOptions.TABLE_KEY, tableName1) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName1); phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") .option(DataSourceOptions.TABLE_KEY, tableName2) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName2); String query = @@ -250,11 +245,11 @@ public void testOrderByWithUnionAll() throws Exception { SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") .option(DataSourceOptions.TABLE_KEY, tableName1) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName1); phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") .option(DataSourceOptions.TABLE_KEY, tableName2) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName2); String query = @@ -300,7 +295,7 @@ public void testCombinationOfOrAndFilters() throws Exception { SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") .option(DataSourceOptions.TABLE_KEY, tableName1) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName1); String dml = "UPSERT INTO " + tableName1 + " VALUES(?,?,?,?,?)"; PreparedStatement stmt = conn.prepareStatement(dml); @@ -391,7 +386,7 @@ public void testOrderByWithExpression() throws Exception { SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") .option(DataSourceOptions.TABLE_KEY, tableName) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName); Dataset dataset = sqlContext.sql("SELECT col1+col2, col4, a_string FROM " + tableName @@ -414,13 +409,14 @@ public void testOrderByWithExpression() throws Exception { @Test public void testColumnFamily() throws Exception { Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); - try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + String jdbcUrl = getUrl(); + try (Connection conn = DriverManager.getConnection(jdbcUrl, props)) { conn.setAutoCommit(false); String tableName = generateUniqueName(); String ddl = "CREATE TABLE " + tableName + " (a_string varchar not null, cf1.a integer, cf1.b varchar, col1 integer, cf2.c varchar, cf2.d integer, col2 integer" + " CONSTRAINT pk PRIMARY KEY (a_string))\n"; - createTestTable(getUrl(), ddl); + createTestTable(jdbcUrl, ddl); String dml = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?,?,?)"; PreparedStatement stmt = conn.prepareStatement(dml); stmt.setString(1, "a"); @@ -453,7 +449,7 @@ public void testColumnFamily() throws Exception { SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") .option(DataSourceOptions.TABLE_KEY, tableName) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.JDBC_URL, jdbcUrl).load(); phoenixDataSet.createOrReplaceTempView(tableName); Dataset dataset = sqlContext.sql("SELECT A_STRING, `CF1.A`, `CF1.B`, COL1, `CF2.C`, `CF2.D`, COL2 from " diff --git a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/SparkResultSet.java b/phoenix5-spark/src/it/java/org/apache/phoenix/spark/SparkResultSet.java similarity index 100% rename from phoenix5-spark/src/main/java/org/apache/phoenix/spark/SparkResultSet.java rename to phoenix5-spark/src/it/java/org/apache/phoenix/spark/SparkResultSet.java diff --git a/phoenix5-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java b/phoenix5-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java index f00dadb2..cdcf4779 100644 --- a/phoenix5-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java +++ b/phoenix5-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java @@ -29,8 +29,6 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.sources.v2.DataSourceOptions; -import scala.Option; -import scala.collection.JavaConverters; import java.sql.Connection; import java.sql.ResultSet; @@ -67,7 +65,7 @@ public static ResultSet executeQuery(Connection conn, QueryBuilder queryBuilder, // since we don't set the predicate filtering is done after rows are returned from spark Dataset phoenixDataSet = getSparkSession().read().format("phoenix") .option(DataSourceOptions.TABLE_KEY, queryBuilder.getFullTableName()) - .option(PhoenixDataSource.ZOOKEEPER_URL, url).load(); + .option(PhoenixDataSource.JDBC_URL, url).load(); phoenixDataSet.createOrReplaceTempView(queryBuilder.getFullTableName()); Dataset dataset = sqlContext.sql(queryBuilder.build()); diff --git a/phoenix5-spark/src/it/resources/globalSetup.sql b/phoenix5-spark/src/it/resources/globalSetup.sql index aa5a81f0..f7f2cec4 100644 --- a/phoenix5-spark/src/it/resources/globalSetup.sql +++ b/phoenix5-spark/src/it/resources/globalSetup.sql @@ -50,9 +50,9 @@ UPSERT INTO TEST_DECIMAL VALUES (1, 123.456789) CREATE TABLE TEST_SMALL_TINY (ID BIGINT NOT NULL PRIMARY KEY, COL1 SMALLINT, COL2 TINYINT) UPSERT INTO TEST_SMALL_TINY VALUES (1, 32767, 127) CREATE TABLE DATE_TEST(ID BIGINT NOT NULL PRIMARY KEY, COL1 DATE) -UPSERT INTO DATE_TEST VALUES(1, CURRENT_DATE()) +UPSERT INTO DATE_TEST VALUES(1, '2021-01-01T00:00:00Z') CREATE TABLE TIME_TEST(ID BIGINT NOT NULL PRIMARY KEY, COL1 TIME) -UPSERT INTO TIME_TEST VALUES(1, CURRENT_TIME()) +UPSERT INTO TIME_TEST VALUES(1, '2021-01-01T00:00:00Z') CREATE TABLE "space" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR) UPSERT INTO "space" VALUES ('key1', 'xyz') CREATE TABLE "small" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR, "salary" INTEGER ) @@ -61,6 +61,7 @@ UPSERT INTO "small" VALUES ('key2', 'bar', 20000) UPSERT INTO "small" VALUES ('key3', 'xyz', 30000) CREATE TABLE MULTITENANT_TEST_TABLE (TENANT_ID VARCHAR NOT NULL, ORGANIZATION_ID VARCHAR, GLOBAL_COL1 VARCHAR CONSTRAINT pk PRIMARY KEY (TENANT_ID, ORGANIZATION_ID)) MULTI_TENANT=true +CREATE TABLE MULTITENANT_TEST_TABLE_WITH_SALT (TENANT_ID VARCHAR NOT NULL, ORGANIZATION_ID VARCHAR, GLOBAL_COL1 VARCHAR CONSTRAINT pk PRIMARY KEY (TENANT_ID, ORGANIZATION_ID)) MULTI_TENANT=true, SALT_BUCKETS = 20 CREATE TABLE IF NOT EXISTS GIGANTIC_TABLE (ID INTEGER PRIMARY KEY,unsig_id UNSIGNED_INT,big_id BIGINT,unsig_long_id UNSIGNED_LONG,tiny_id TINYINT,unsig_tiny_id UNSIGNED_TINYINT,small_id SMALLINT,unsig_small_id UNSIGNED_SMALLINT,float_id FLOAT,unsig_float_id UNSIGNED_FLOAT,double_id DOUBLE,unsig_double_id UNSIGNED_DOUBLE,decimal_id DECIMAL,boolean_id BOOLEAN,time_id TIME,date_id DATE,timestamp_id TIMESTAMP,unsig_time_id UNSIGNED_TIME,unsig_date_id UNSIGNED_DATE,unsig_timestamp_id UNSIGNED_TIMESTAMP,varchar_id VARCHAR (30),char_id CHAR (30),binary_id BINARY (100),varbinary_id VARBINARY (100)) CREATE TABLE IF NOT EXISTS OUTPUT_GIGANTIC_TABLE (ID INTEGER PRIMARY KEY,unsig_id UNSIGNED_INT,big_id BIGINT,unsig_long_id UNSIGNED_LONG,tiny_id TINYINT,unsig_tiny_id UNSIGNED_TINYINT,small_id SMALLINT,unsig_small_id UNSIGNED_SMALLINT,float_id FLOAT,unsig_float_id UNSIGNED_FLOAT,double_id DOUBLE,unsig_double_id UNSIGNED_DOUBLE,decimal_id DECIMAL,boolean_id BOOLEAN,time_id TIME,date_id DATE,timestamp_id TIMESTAMP,unsig_time_id UNSIGNED_TIME,unsig_date_id UNSIGNED_DATE,unsig_timestamp_id UNSIGNED_TIMESTAMP,varchar_id VARCHAR (30),char_id CHAR (30),binary_id BINARY (100),varbinary_id VARBINARY (100)) UPSERT INTO GIGANTIC_TABLE VALUES(0,2,3,4,-5,6,7,8,9.3,10.4,11.5,12.6,13.7,true,null,null,CURRENT_TIME(),CURRENT_TIME(),CURRENT_DATE(),CURRENT_TIME(),'This is random textA','a','a','a') diff --git a/phoenix5-spark/src/it/resources/tenantSetup.sql b/phoenix5-spark/src/it/resources/tenantSetup.sql index f62d8430..752dbbd4 100644 --- a/phoenix5-spark/src/it/resources/tenantSetup.sql +++ b/phoenix5-spark/src/it/resources/tenantSetup.sql @@ -16,3 +16,5 @@ CREATE VIEW IF NOT EXISTS TENANT_VIEW(TENANT_ONLY_COL VARCHAR) AS SELECT * FROM MULTITENANT_TEST_TABLE UPSERT INTO TENANT_VIEW (ORGANIZATION_ID, TENANT_ONLY_COL) VALUES ('defaultOrg', 'defaultData') +CREATE VIEW IF NOT EXISTS TENANT_VIEW_WITH_SALT(TENANT_ONLY_COL VARCHAR) AS SELECT * FROM MULTITENANT_TEST_TABLE_WITH_SALT +UPSERT INTO TENANT_VIEW_WITH_SALT (ORGANIZATION_ID, TENANT_ONLY_COL) VALUES ('defaultOrg', 'defaultData') diff --git a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala index 1b244034..776cc77f 100644 --- a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala @@ -14,15 +14,14 @@ package org.apache.phoenix.spark import java.sql.{Connection, DriverManager} -import java.util.Properties - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; +import java.util.{Properties, TimeZone} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hbase.HConstants import org.apache.phoenix.query.BaseTest +import org.apache.phoenix.spark.datasource.v2.writer.PhoenixTestingDataSourceWriter import org.apache.phoenix.util.PhoenixRuntime -import org.apache.phoenix.util.ReadOnlyProps; -import org.apache.spark.sql.{SQLContext, SparkSession} -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.phoenix.util.ReadOnlyProps +import org.apache.spark.sql.SparkSession import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite, Matchers} @@ -31,6 +30,8 @@ object PhoenixSparkITHelper extends BaseTest { def getTestClusterConfig = new Configuration(BaseTest.config); def doSetup = { + // Set-up fixed timezone for DATE and TIMESTAMP tests + TimeZone.setDefault(TimeZone.getTimeZone("UTC")) // The @ClassRule doesn't seem to be getting picked up, force creation here before setup BaseTest.tmpFolder.create() BaseTest.setUpTestDriver(ReadOnlyProps.EMPTY_PROPS); @@ -64,9 +65,9 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter conf } - lazy val quorumAddress = { - ConfigurationUtil.getZookeeperURL(hbaseConfiguration).get - } + lazy val jdbcUrl = PhoenixSparkITHelper.getUrl + + lazy val quorumAddress = ConfigurationUtil.getZookeeperURL(hbaseConfiguration).get // Runs SQL commands located in the file defined in the sqlSource argument // Optional argument tenantId used for running tenant-specific SQL @@ -93,6 +94,11 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter conn.commit() } + before{ + // Reset batch counter after each test + PhoenixTestingDataSourceWriter.TOTAL_BATCHES_COMMITTED_COUNT = 0 + } + override def beforeAll() { PhoenixSparkITHelper.doSetup @@ -103,12 +109,6 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter // We pass in a TenantId to allow the DDL to create tenant-specific tables/views setupTables("tenantSetup.sql", Some(TenantId)) - //FIXME is this ever used ? - val conf = new SparkConf() - .setAppName("PhoenixSparkIT") - .setMaster("local[2]") // 2 threads, some parallelism - .set("spark.ui.showConsoleProgress", "false") // Disable printing stage progress - spark = SparkSession .builder() .appName("PhoenixSparkIT") diff --git a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala new file mode 100644 index 00000000..0450ce05 --- /dev/null +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala @@ -0,0 +1,743 @@ +/* + Licensed 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.phoenix.spark + +import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil +import org.apache.phoenix.schema.types.{PSmallintArray, PVarchar} +import org.apache.phoenix.spark.datasource.v2.reader.PhoenixTestingInputPartitionReader +import org.apache.phoenix.spark.datasource.v2.writer.PhoenixTestingDataSourceWriter +import org.apache.phoenix.spark.datasource.v2.{PhoenixDataSource, PhoenixTestingDataSource} +import org.apache.phoenix.util.{ColumnInfo, SchemaUtil} +import org.apache.spark.SparkException +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Row, SaveMode} + +import java.sql.DriverManager +import java.time.{LocalDate, ZoneId} +import java.util.Date +import scala.collection.mutable +import scala.collection.mutable.ListBuffer + +/** + * Note: If running directly from an IDE, these are the recommended VM parameters: + * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + */ +class PhoenixSparkDatasourceV1IT extends AbstractPhoenixSparkIT { + + test("Can persist data with case sensitive columns (like in avro schema)") { + val df = spark.createDataFrame( + Seq( + (1, 1, "test_child_1"), + (2, 1, "test_child_2"))). + // column names are case sensitive + toDF("ID", "TABLE3_ID", "t2col1") + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "TABLE3", + PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, + PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true")) + .mode(SaveMode.Overwrite) + .save() + + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3") + + val checkResults = List((1, 1, "test_child_1"), (2, 1, "test_child_2")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + + test("Can convert Phoenix schema") { + val phoenixSchema = List( + new ColumnInfo("varcharColumn", PVarchar.INSTANCE.getSqlType) + ) + + val catalystSchema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(phoenixSchema) + + val expected = new StructType(List(StructField("varcharColumn", StringType, nullable = true)).toArray) + + catalystSchema shouldEqual expected + } + + test("Can convert arrays of Short type in Phoenix schema") { + val phoenixSchema = List( + new ColumnInfo("arrayshortColumn", PSmallintArray.INSTANCE.getSqlType) + ) + + val catalystSchema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(phoenixSchema) + + val expected = new StructType(List(StructField("arrayshortColumn", ArrayType(ShortType, true), nullable = true)).toArray) + + catalystSchema shouldEqual expected + } + + test("Can create schema RDD and execute query") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("sql_table_1") + + val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df2.createOrReplaceTempView("sql_table_2") + + val sqlRdd = spark.sql( + """ + |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 + |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin + ) + + val count = sqlRdd.count() + + count shouldEqual 6L + } + + ignore("Ordering by pk columns should not require sorting") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + df1.createOrReplaceTempView("TABLE1") + + val sqlRdd = spark.sql("SELECT * FROM TABLE1 ORDER BY ID, COL1") + val plan = sqlRdd.queryExecution.sparkPlan + // verify the spark plan doesn't have a sort + assert(!plan.toString.contains("Sort")) + + val expectedResults = Array(Row.fromSeq(Seq(1, "test_row_1")), Row.fromSeq(Seq(2, "test_row_2"))) + val actual = sqlRdd.collect() + + actual shouldEqual expectedResults + } + + test("Verify correct number of partitions are created") { + val conn = DriverManager.getConnection(PhoenixSparkITHelper.getUrl) + val ddl = "CREATE TABLE SPLIT_TABLE (id VARCHAR NOT NULL PRIMARY KEY, val VARCHAR) split on ('e','j','o')" + conn.createStatement.execute(ddl) + val keys = Array("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s", + "t", "u", "v", "w", "x", "y", "z") + for (key <- keys) { + conn.createStatement.execute("UPSERT INTO SPLIT_TABLE VALUES('" + key + "', '" + key + "')") + } + conn.commit() + + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "SPLIT_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + df1.createOrReplaceTempView("SPLIT_TABLE") + val sqlRdd = spark.sql("SELECT * FROM SPLIT_TABLE") + val numPartitions = sqlRdd.rdd.partitions.size + + numPartitions shouldEqual 4 + } + + test("Can create schema RDD and execute query on case sensitive table (no config)") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> SchemaUtil.getEscapedArgument("table4"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("table4") + + val sqlRdd = spark.sql("SELECT id FROM table4") + + val count = sqlRdd.count() + + count shouldEqual 2L + } + + test("Can create schema RDD and execute constrained query") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("sql_table_1") + + val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = 1") + + df2.createOrReplaceTempView("sql_table_2") + + val sqlRdd = spark.sql( + """ + |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 + |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin + ) + + val count = sqlRdd.count() + + count shouldEqual 1L + } + + test("Can create schema RDD with predicate that will never match") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = -1") + + df1.createOrReplaceTempView("table3") + + val sqlRdd = spark.sql("SELECT * FROM table3") + + val count = sqlRdd.count() + + count shouldEqual 0L + } + + test("Can create schema RDD with complex predicate") { + val predicate = "ID > 0 AND TIMESERIES_KEY BETWEEN " + + "CAST(TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP) AND " + + "CAST(TO_DATE('1990-01-30 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP)" + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> "DATE_PREDICATE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + .filter(predicate) + + df1.createOrReplaceTempView("date_predicate_test_table") + + val sqlRdd = spark.sqlContext.sql("SELECT * FROM date_predicate_test_table") + + val count = sqlRdd.count() + + count shouldEqual 0L + } + + test("Can query an array table") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("ARRAY_TEST_TABLE") + + val sqlRdd = spark.sql("SELECT * FROM ARRAY_TEST_TABLE") + + val count = sqlRdd.count() + + // get row 0, column 1, which should be "VCARRAY" + val arrayValues = sqlRdd.collect().apply(0).apply(1) + + arrayValues should equal(Array("String1", "String2", "String3")) + + count shouldEqual 1L + } + + test("Can read a table as an RDD") { + val rdd1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + val count = rdd1.count() + + val arrayValues = rdd1.take(1)(0)(1) + + arrayValues should equal(Array("String1", "String2", "String3")) + + count shouldEqual 1L + } + + test("Can save to phoenix table from Spark without specifying all the columns") { + val dataSet = List(Row(1L, "1", 1), Row(2L, "2", 2), Row(3L, "3", 3)) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("COL1", StringType), + StructField("COL2", IntegerType))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT ID, COL1, COL2 FROM OUTPUT_TEST_TABLE") + val results = ListBuffer[Row]() + while (rs.next()) { + results.append(Row(rs.getLong(1), rs.getString(2), rs.getInt(3))) + } + + // Verify they match + (0 to results.size - 1).foreach { i => + dataSet(i) shouldEqual results(i) + } + } + + test("Can save dates to Phoenix using java.sql.Date") { + val date = java.sql.Date.valueOf("2016-09-30") + + // Since we are creating a Row we have to use java.sql.date + // java.util.date or joda.DateTime is not supported + val dataSet = Seq(Row(1L, "1", 1, date), Row(2L, "2", 2, date)) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("COL1", StringType), + StructField("COL2", IntegerType), + StructField("COL3", DateType))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT COL3 FROM OUTPUT_TEST_TABLE WHERE ID = 1 OR ID = 2 ORDER BY ID ASC") + val results = ListBuffer[java.sql.Date]() + while (rs.next()) { + results.append(rs.getDate(1)) + } + + // Verify the epochs are equal + results(0).getTime shouldEqual date.getTime + results(1).getTime shouldEqual date.getTime + } + + test("Can infer schema without defining columns") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + df.schema("ID").dataType shouldEqual LongType + df.schema("TABLE1_ID").dataType shouldEqual LongType + df.schema("t2col1").dataType shouldEqual StringType + } + + test("Spark SQL can use Phoenix as a data source with no schema specified") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + df.count() shouldEqual 2 + df.schema("ID").dataType shouldEqual LongType + df.schema("COL1").dataType shouldEqual StringType + } + + + test("Can persist a dataframe") { + // Load from TABLE1 + val df = spark.sqlContext.read.format("org.apache.phoenix.spark").options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + // Save to TABLE1_COPY + df + .write + .format("org.apache.phoenix.spark") + .mode(SaveMode.Overwrite) + .option("table", "TABLE1_COPY") + .option(PhoenixDataSource.ZOOKEEPER_URL, quorumAddress) + .save() + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE1_COPY") + + val checkResults = List((1L, "test_row_1"), (2, "test_row_2")) + val results = ListBuffer[(Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getString(2))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + + test("Can save arrays back to phoenix") { + val dataSet = List(Row(2L, Array("String1", "String2", "String3"))) + val schema = StructType(Seq( + StructField("ID", LongType, nullable = false), + StructField("VCARRAY", ArrayType(StringType, true)) + )) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT VCARRAY FROM ARRAY_TEST_TABLE WHERE ID = 2") + rs.next() + val sqlArray = rs.getArray(1).getArray().asInstanceOf[Array[String]] + + // Verify the arrays are equal + sqlArray shouldEqual dataSet(0).get(1) + } + + test("Can read from table with schema and escaped table name") { + // Manually escape + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + + var count = df1.count() + + count shouldEqual 1L + + // Use SchemaUtil + val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( + Map("table" -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load() + + count = df2.count() + + count shouldEqual 1L + } + + test("Ensure DataFrame field normalization (PHOENIX-2196)") { + val rdd1 = spark.sparkContext + .parallelize(Seq((1L, 1L, "One"), (2L, 2L, "Two"))) + .map(p => Row(p._1, p._2, p._3)) + + val schema = StructType(Seq( + StructField("id", LongType, nullable = false), + StructField("table1_id", LongType, nullable = true), + StructField("\"t2col1\"", StringType, nullable = true) + )) + + val df = spark.sqlContext.createDataFrame(rdd1, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + } + + test("Ensure Dataframe supports LIKE and IN filters (PHOENIX-2328)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark").options(Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + // Prefix match + val res1 = df.filter("COL1 like 'test_row_%'") + val plan = res1.groupBy().count().queryExecution.sparkPlan + res1.count() shouldEqual 2 + + // Suffix match + val res2 = df.filter("COL1 like '%_1'") + res2.count() shouldEqual 1 + res2.first.getString(1) shouldEqual "test_row_1" + + // Infix match + val res3 = df.filter("COL1 like '%_row_%'") + res3.count() shouldEqual 2 + + // Not like, match none + val res4 = df.filter("COL1 not like '%_row_%'") + res4.count() shouldEqual 0 + + // Not like, match all + val res5 = df.filter("COL1 not like '%_wor_%'") + res5.count() shouldEqual 2 + + // "IN", match all + val res6 = df.filter("COL1 in ('test_row_1', 'test_row_2')") + res6.count() shouldEqual 2 + + // "IN", match none + val res7 = df.filter("COL1 in ('foo', 'bar')") + res7.count() shouldEqual 0 + + // AND (and not again) + val res8 = df.filter("COL1 like '%_row_%' AND COL1 not like '%_1'") + res8.count() shouldEqual 1 + res8.first.getString(1) shouldEqual "test_row_2" + + // OR + val res9 = df.filter("COL1 like '%_1' OR COL1 like '%_2'") + res9.count() shouldEqual 2 + } + + test("Can load decimal types with accurate precision and scale (PHOENIX-2288)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> "TEST_DECIMAL", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + assert(df.select("COL1").first().getDecimal(0) == BigDecimal("123.456789").bigDecimal) + } + + test("Can load small and tiny integer types (PHOENIX-2426)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> "TEST_SMALL_TINY", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + assert(df.select("COL1").first().getShort(0).toInt == 32767) + assert(df.select("COL2").first().getByte(0).toInt == 127) + } + + test("Can save arrays from custom dataframes back to phoenix") { + val dataSet = List(Row(2L, Array("String1", "String2", "String3"), Array(1, 2, 3))) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("VCARRAY", ArrayType(StringType)), + StructField("INTARRAY", ArrayType(IntegerType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "ARRAYBUFFER_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT VCARRAY, INTARRAY FROM ARRAYBUFFER_TEST_TABLE WHERE ID = 2") + rs.next() + val stringArray = rs.getArray(1).getArray().asInstanceOf[Array[String]] + val intArray = rs.getArray(2).getArray().asInstanceOf[Array[Int]] + + // Verify the arrays are equal + stringArray shouldEqual dataSet(0).getAs[Array[String]](1) + intArray shouldEqual dataSet(0).getAs[Array[Int]](2) + } + + test("Can save arrays of AnyVal type back to phoenix") { + val dataSet = List(Row(2L, Array(1, 2, 3), Array(1L, 2L, 3L))) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("INTARRAY", ArrayType(IntegerType)), + StructField("BIGINTARRAY", ArrayType(LongType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "ARRAY_ANYVAL_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT INTARRAY, BIGINTARRAY FROM ARRAY_ANYVAL_TEST_TABLE WHERE ID = 2") + rs.next() + val intArray = rs.getArray(1).getArray().asInstanceOf[Array[Int]] + val longArray = rs.getArray(2).getArray().asInstanceOf[Array[Long]] + + // Verify the arrays are equal + intArray shouldEqual dataSet(0).get(1) + longArray shouldEqual dataSet(0).get(2) + } + + test("Can save arrays of Byte type back to phoenix") { + val dataSet = List(Row(2L, Array(1.toByte, 2.toByte, 3.toByte))) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("BYTEARRAY", ArrayType(ByteType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "ARRAY_BYTE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT BYTEARRAY FROM ARRAY_BYTE_TEST_TABLE WHERE ID = 2") + rs.next() + val byteArray = rs.getArray(1).getArray().asInstanceOf[Array[Byte]] + + // Verify the arrays are equal + byteArray shouldEqual dataSet(0).get(1) + } + + test("Can save arrays of Short type back to phoenix") { + val dataSet = List(Row(2L, Array(1.toShort, 2.toShort, 3.toShort))) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("SHORTARRAY", ArrayType(ShortType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "ARRAY_SHORT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT SHORTARRAY FROM ARRAY_SHORT_TEST_TABLE WHERE ID = 2") + rs.next() + val shortArray = rs.getArray(1).getArray().asInstanceOf[Array[Short]] + + // Verify the arrays are equal + shortArray shouldEqual dataSet(0).get(1) + } + + test("Can save binary types back to phoenix") { + val dataSet = List(Row(2L, Array[Byte](1), Array[Byte](1, 2, 3), Array[Array[Byte]](Array[Byte](1), Array[Byte](2)))) + + val schema = StructType( + Seq(StructField("ID", LongType, false), + StructField("BIN", BinaryType), + StructField("VARBIN", BinaryType), + StructField("BINARRAY", ArrayType(BinaryType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "VARBINARY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT BIN, VARBIN, BINARRAY FROM VARBINARY_TEST_TABLE WHERE ID = 2") + rs.next() + val byte = rs.getBytes("BIN") + val varByte = rs.getBytes("VARBIN") + val varByteArray = rs.getArray("BINARRAY").getArray().asInstanceOf[Array[Array[Byte]]] + + // Verify the arrays are equal + byte shouldEqual dataSet(0).get(1) + varByte shouldEqual dataSet(0).get(2) + varByteArray shouldEqual dataSet(0).get(3) + } + + test("Can load and filter Phoenix DATE columns through DataFrame API") { + val df = spark.sqlContext.read + .format("org.apache.phoenix.spark") + .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + val dt = df.select("COL1").first().getDate(0).getTime + val expected = LocalDate.of(2021,1,1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli + + // NOTE: Spark DateType drops hour, minute, second, as per the java.sql.Date spec + // Use 'dateAsTimestamp' option to coerce DATE to TIMESTAMP without losing resolution + + // Note that Spark also applies the timezone offset to the returned date epoch. Rather than perform timezone + // gymnastics, just make sure we're within 24H of the epoch generated just now + assert(expected == dt) + + df.createOrReplaceTempView("DATE_TEST") + val df2 = spark.sql("SELECT * FROM DATE_TEST WHERE COL1 > TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss')") + assert(df2.count() == 1L) + } + + test("Filter operation doesn't work for column names containing a white space (PHOENIX-2547)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> SchemaUtil.getEscapedArgument("space"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + val res = df.filter(df.col("first name").equalTo("xyz")) + // Make sure we got the right value back + assert(res.collectAsList().size() == 1L) + } + + test("Spark Phoenix cannot recognize Phoenix view fields (PHOENIX-2290)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + df.createOrReplaceTempView("temp") + + // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions + // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle + + val res1 = spark.sql("select * from temp where salary = '10000' ") + assert(res1.collectAsList().size() == 1L) + + val res2 = spark.sql("select * from temp where \"salary\" = '10000' ") + assert(res2.collectAsList().size() == 0L) + + val res3 = spark.sql("select * from temp where salary > '10000' ") + assert(res3.collectAsList().size() == 2L) + } + + test("Queries with small case column-names return empty result-set when working with Spark Datasource Plugin (PHOENIX-2336)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + + // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions + // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle + + val res1 = df.filter(df.col("first name").equalTo("foo")) + assert(res1.collectAsList().size() == 1L) + + val res2 = df.filter("\"first name\" = 'foo'") + assert(res2.collectAsList().size() == 0L) + + val res3 = df.filter("salary = '10000'") + assert(res3.collectAsList().size() == 1L) + + val res4 = df.filter("salary > '10000'") + assert(res4.collectAsList().size() == 2L) + } + + test("Can coerce Phoenix DATE columns to TIMESTAMP through DataFrame API") { + val df = spark.sqlContext.read + .format("org.apache.phoenix.spark") + .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, "dateAsTimestamp" -> "true")) + .load + val dtRes = df.select("COL1").first() + val ts = dtRes.getTimestamp(0).getTime + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli + + assert(expected == ts) + } + + test("Can load Phoenix Time columns through DataFrame API") { + val df = spark.sqlContext.read + .format("org.apache.phoenix.spark") + .options(Map("table" -> "TIME_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + val time = df.select("COL1").first().getTimestamp(0).getTime + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli + + assert(expected == time) + } + + test("can read all Phoenix data types") { + val df = spark.sqlContext.read + .format("org.apache.phoenix.spark") + .options(Map("table" -> "GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + + df.write + .format("org.apache.phoenix.spark") + .options(Map("table" -> "OUTPUT_GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + df.count() shouldEqual 1 + } + +} diff --git a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala new file mode 100644 index 00000000..28907dbc --- /dev/null +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala @@ -0,0 +1,117 @@ +/* + Licensed 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.phoenix.spark + +import scala.collection.mutable.ListBuffer + +/** + * Sub-class of PhoenixSparkIT used for tenant-specific tests + * + * Note: All schema related variables (table name, column names, default data, etc) are coupled with + * phoenix-spark/src/it/resources/tenantSetup.sql + * + * Note: If running directly from an IDE, these are the recommended VM parameters: + * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + * + */ +class PhoenixSparkDatasourceV1ITTenantSpecific extends AbstractPhoenixSparkIT { + + // Tenant-specific schema info + val OrgIdCol = "ORGANIZATION_ID" + val TenantOnlyCol = "TENANT_ONLY_COL" + val TenantTable = "TENANT_VIEW" + + // Data set for tests that write to Phoenix + val TestDataSet = List(("testOrg1", "data1"), ("testOrg2", "data2"), ("testOrg3", "data3")) + val TestDataSet2 = List(("testOrg1", "data1", TenantId, "g1"), ("testOrg2", "data2", TenantId, "g3"), + ("testOrg3", "data3", TenantId, "g3")) + + /** + * Helper method used by write tests to verify content written. + * Assumes the caller has written the TestDataSet (defined above) to Phoenix + * and that 1 row of default data exists (upserted after table creation in tenantSetup.sql) + */ + def verifyResults(): Unit = { + // Contains the default data upserted into the tenant-specific table in tenantSetup.sql and the data upserted by tests + val VerificationDataSet = List(("defaultOrg", "defaultData")) ::: TestDataSet + + val SelectStatement = "SELECT " + OrgIdCol + "," + TenantOnlyCol + " FROM " + TenantTable + val stmt = conn.createStatement() + val rs = stmt.executeQuery(SelectStatement) + + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual VerificationDataSet + } + + /*****************/ + /** Read tests **/ + /*****************/ + + test("Can read from tenant-specific table as DataFrame") { + val df = spark.sqlContext.phoenixTableAsDataFrame( + TenantTable, + Seq(OrgIdCol, TenantOnlyCol), + zkUrl = Some(quorumAddress), + tenantId = Some(TenantId), + conf = hbaseConfiguration) + + // There should only be 1 row upserted in tenantSetup.sql + val count = df.count() + count shouldEqual 1L + } + + test("Can read from tenant-specific table as RDD") { + val rdd = spark.sparkContext.phoenixTableAsRDD( + TenantTable, + Seq(OrgIdCol, TenantOnlyCol), + zkUrl = Some(quorumAddress), + tenantId = Some(TenantId), + conf = hbaseConfiguration) + + // There should only be 1 row upserted in tenantSetup.sql + val count = rdd.count() + count shouldEqual 1L + } + + /*****************/ + /** Write tests **/ + /*****************/ + + test("Can write a DataFrame using 'DataFrame.saveToPhoenix' to tenant-specific view") { + val sqlContext = spark.sqlContext + import sqlContext.implicits._ + + val df = spark.sparkContext.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol) + df.saveToPhoenix(TenantTable, zkUrl = Some(quorumAddress), tenantId = Some(TenantId)) + + verifyResults + } + + test("Can write an RDD to Phoenix tenant-specific view") { + spark.sparkContext + .parallelize(TestDataSet) + .saveToPhoenix( + TenantTable, + Seq(OrgIdCol, TenantOnlyCol), + hbaseConfiguration, + tenantId = Some(TenantId) + ) + + verifyResults + } +} diff --git a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala index f1a09d64..b9c6052d 100644 --- a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala @@ -13,13 +13,12 @@ */ package org.apache.phoenix.spark -import org.apache.omid.tso.client.AbortException import java.sql.DriverManager import java.util.Date import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil import org.apache.phoenix.query.QueryServices -import org.apache.phoenix.schema.types.{PSmallintArray, PUnsignedSmallintArray, PVarchar} +import org.apache.phoenix.schema.types.{PSmallintArray, PVarchar} import org.apache.phoenix.spark.datasource.v2.{PhoenixDataSource, PhoenixTestingDataSource} import org.apache.phoenix.spark.datasource.v2.reader.PhoenixTestingInputPartitionReader import org.apache.phoenix.spark.datasource.v2.writer.PhoenixTestingDataSourceWriter @@ -28,26 +27,28 @@ import org.apache.spark.SparkException import org.apache.spark.sql.types.{ArrayType, BinaryType, ByteType, DateType, IntegerType, LongType, ShortType, StringType, StructField, StructType} import org.apache.spark.sql.{Row, SaveMode} +import java.time.{Instant, LocalDate, ZoneId, ZoneOffset} import scala.collection.mutable import scala.collection.mutable.ListBuffer /** - * Note: If running directly from an IDE, these are the recommended VM parameters: - * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m - */ + * Note: If running directly from an IDE, these are the recommended VM parameters: + * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + */ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can persist data with case sensitive columns (like in avro schema)") { val df = spark.createDataFrame( - Seq( - (1, 1, "test_child_1"), - (2, 1, "test_child_2"))). - // column names are case sensitive + Seq( + (1, 1, "test_child_1"), + (2, 1, "test_child_2"))). + // column names are case sensitive toDF("ID", "TABLE3_ID", "t2col1") df.write .format("phoenix") .options(Map("table" -> "TABLE3", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true")) + PhoenixDataSource.JDBC_URL -> jdbcUrl, + PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true")) .mode(SaveMode.Overwrite) .save() @@ -66,41 +67,19 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { results.toList shouldEqual checkResults } - // INSERT is not support using DataSource v2 api yet - ignore("Can use write data using spark SQL INSERT") { - val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE3", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - df1.createOrReplaceTempView("TABLE3") - - // Insert data - spark.sql("INSERT INTO TABLE3 VALUES(10, 10, 10)") - spark.sql("INSERT INTO TABLE3 VALUES(20, 20, 20)") - - // Verify results - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT * FROM TABLE3 WHERE ID>=10") - val expectedResults = List((10, 10, "10"), (20, 20, "20")) - val results = ListBuffer[(Long, Long, String)]() - while (rs.next()) { - results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) - } - stmt.close() - - results.toList shouldEqual expectedResults - } - test("Can persist data into transactional tables with phoenix.transactions.enabled option") { var extraOptions = QueryServices.TRANSACTIONS_ENABLED + "=true"; val df = spark.createDataFrame( - Seq( - (1, 1, "test_child_1"), - (2, 1, "test_child_2"))). + Seq( + (1, 1, "test_child_1"), + (2, 1, "test_child_2"))). // column names are case sensitive toDF("ID", "TABLE5_ID", "t5col1") df.write .format("phoenix") .options(Map("table" -> "TABLE5", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true", + PhoenixDataSource.JDBC_URL -> jdbcUrl, + PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true", PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)) .mode(SaveMode.Overwrite) .save() @@ -128,15 +107,16 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { var extraOptions = QueryServices.TRANSACTIONS_ENABLED + "=true" val df = spark.createDataFrame( - Seq( - (1, 1, "test_child_1"), - (2, 1, "test_child_2"))). + Seq( + (1, 1, "test_child_1"), + (2, 1, "test_child_2"))). // column names are case sensitive toDF("ID", "TABLE5_ID", "t5col1") df.write .format("phoenix") .options(Map("table" -> "TABLE5", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true", + PhoenixDataSource.JDBC_URL -> jdbcUrl, + PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true", PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)) .mode(SaveMode.Overwrite) .save() @@ -186,12 +166,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can create schema RDD and execute query") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("sql_table_1") val df2 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> "TABLE2", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df2.createOrReplaceTempView("sql_table_2") @@ -208,7 +188,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { ignore("Ordering by pk columns should not require sorting") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("TABLE1") val sqlRdd = spark.sql("SELECT * FROM TABLE1 ORDER BY ID, COL1") @@ -234,7 +214,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { conn.commit() val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "SPLIT_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> "SPLIT_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("SPLIT_TABLE") val sqlRdd = spark.sql("SELECT * FROM SPLIT_TABLE") val numPartitions = sqlRdd.rdd.partitions.size @@ -244,7 +224,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can create schema RDD and execute query on case sensitive table (no config)") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> SchemaUtil.getEscapedArgument("table4"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> SchemaUtil.getEscapedArgument("table4"), PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("table4") @@ -257,12 +237,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can create schema RDD and execute constrained query") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("sql_table_1") val df2 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = 1") + .options(Map("table" -> "TABLE2", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load.filter("ID = 1") df2.createOrReplaceTempView("sql_table_2") @@ -279,7 +259,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can create schema RDD with predicate that will never match") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = -1") + .options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load.filter("ID = -1") df1.createOrReplaceTempView("table3") @@ -295,7 +275,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { "CAST(TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP) AND " + "CAST(TO_DATE('1990-01-30 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP)" val df1 = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> "DATE_PREDICATE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "DATE_PREDICATE_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load .filter(predicate) @@ -310,7 +290,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can query an array table") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("ARRAY_TEST_TABLE") @@ -328,7 +308,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can read a table as an RDD") { val rdd1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load val count = rdd1.count() @@ -344,7 +324,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { var extraOptions = PhoenixTestingInputPartitionReader.RETURN_NULL_CURR_ROW + "=true" var rdd = spark.sqlContext.read .format(PhoenixTestingDataSource.TEST_SOURCE) - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, + .options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl, PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)).load // Expect to get a NullPointerException in the executors @@ -357,7 +337,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { extraOptions = PhoenixTestingInputPartitionReader.RETURN_NULL_CURR_ROW + "=false" rdd = spark.sqlContext.read .format(PhoenixTestingDataSource.TEST_SOURCE) - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, + .options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl, PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)).load val stringValue = rdd.take(2)(0)(1) stringValue shouldEqual "test_row_1" @@ -378,7 +358,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() @@ -422,19 +402,19 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { // Apply the schema to the RDD. val df = spark.sqlContext.createDataFrame(rowRDD, schema) - val extraOptions = PhoenixConfigurationUtil.UPSERT_BATCH_SIZE + "=" + upsertBatchSize.toString + val extraOptions = PhoenixConfigurationUtil.UPSERT_BATCH_SIZE + "=" + upsertBatchSize.toString // Initially, this should be zero PhoenixTestingDataSourceWriter.TOTAL_BATCHES_COMMITTED_COUNT shouldEqual 0 df.write .format(PhoenixTestingDataSource.TEST_SOURCE) - .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, + .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl, PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)) .mode(SaveMode.Overwrite) .save() // Verify the number of times batched updates are committed via DataWriters - PhoenixTestingDataSourceWriter.TOTAL_BATCHES_COMMITTED_COUNT shouldEqual totalRecords/upsertBatchSize + PhoenixTestingDataSourceWriter.TOTAL_BATCHES_COMMITTED_COUNT shouldEqual totalRecords / upsertBatchSize } test("Can save dates to Phoenix using java.sql.Date") { @@ -457,7 +437,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() @@ -476,7 +456,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can infer schema without defining columns") { val df = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + .options(Map("table" -> "TABLE2", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() df.schema("ID").dataType shouldEqual LongType df.schema("TABLE1_ID").dataType shouldEqual LongType df.schema("t2col1").dataType shouldEqual StringType @@ -484,7 +464,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Spark SQL can use Phoenix as a data source with no schema specified") { val df = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df.count() shouldEqual 2 df.schema("ID").dataType shouldEqual LongType df.schema("COL1").dataType shouldEqual StringType @@ -492,7 +472,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Datasource v2 pushes down filters") { val df = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load val res = df.filter(df("COL1") === "test_row_1" && df("ID") === 1L).select(df("ID")) // Make sure we got the right value back @@ -507,7 +487,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can persist a dataframe") { // Load from TABLE1 - val df = spark.sqlContext.read.format("phoenix").options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + val df = spark.sqlContext.read.format("phoenix").options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load // Save to TABLE1_COPY df @@ -515,7 +495,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { .format("phoenix") .mode(SaveMode.Overwrite) .option("table", "TABLE1_COPY") - .option(PhoenixDataSource.ZOOKEEPER_URL, quorumAddress) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) .save() // Verify results @@ -546,7 +526,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() @@ -563,7 +543,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can read from table with schema and escaped table name") { // Manually escape val df1 = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + .options(Map("table" -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() var count = df1.count() @@ -572,7 +552,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { // Use SchemaUtil val df2 = spark.sqlContext.read.format("phoenix") .options( - Map("table" -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + Map("table" -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load() count = df2.count() @@ -595,13 +575,13 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "TABLE2", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() } test("Ensure Dataframe supports LIKE and IN filters (PHOENIX-2328)") { - val df = spark.sqlContext.read.format("phoenix").options(Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + val df = spark.sqlContext.read.format("phoenix").options(Map("table" -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() // Prefix match val res1 = df.filter("COL1 like 'test_row_%'") val plan = res1.groupBy().count().queryExecution.sparkPlan @@ -644,13 +624,13 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can load decimal types with accurate precision and scale (PHOENIX-2288)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> "TEST_DECIMAL", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + .options(Map("table" -> "TEST_DECIMAL", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() assert(df.select("COL1").first().getDecimal(0) == BigDecimal("123.456789").bigDecimal) } test("Can load small and tiny integer types (PHOENIX-2426)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> "TEST_SMALL_TINY", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + .options(Map("table" -> "TEST_SMALL_TINY", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() assert(df.select("COL1").first().getShort(0).toInt == 32767) assert(df.select("COL2").first().getByte(0).toInt == 127) } @@ -670,7 +650,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAYBUFFER_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "ARRAYBUFFER_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() @@ -701,7 +681,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAY_ANYVAL_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "ARRAY_ANYVAL_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() @@ -731,7 +711,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAY_BYTE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "ARRAY_BYTE_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() @@ -759,7 +739,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAY_SHORT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "ARRAY_SHORT_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() @@ -789,7 +769,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "VARBINARY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "VARBINARY_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() @@ -810,17 +790,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can load and filter Phoenix DATE columns through DataFrame API") { val df = spark.sqlContext.read .format("phoenix") - .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "DATE_TEST", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load val dt = df.select("COL1").first().getDate(0).getTime - val epoch = new Date().getTime + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - // NOTE: Spark DateType drops hour, minute, second, as per the java.sql.Date spec - // Use 'dateAsTimestamp' option to coerce DATE to TIMESTAMP without losing resolution - - // Note that Spark also applies the timezone offset to the returned date epoch. Rather than perform timezone - // gymnastics, just make sure we're within 24H of the epoch generated just now - assert(Math.abs(epoch - dt) < 86400000) + assert(expected == dt) df.createOrReplaceTempView("DATE_TEST") val df2 = spark.sql("SELECT * FROM DATE_TEST WHERE COL1 > TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss')") @@ -829,7 +804,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Filter operation doesn't work for column names containing a white space (PHOENIX-2547)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> SchemaUtil.getEscapedArgument("space"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> SchemaUtil.getEscapedArgument("space"), PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load val res = df.filter(df.col("first name").equalTo("xyz")) // Make sure we got the right value back @@ -838,7 +813,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Spark Phoenix cannot recognize Phoenix view fields (PHOENIX-2290)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load df.createOrReplaceTempView("temp") @@ -857,7 +832,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Queries with small case column-names return empty result-set when working with Spark Datasource Plugin (PHOENIX-2336)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions @@ -879,34 +854,35 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can coerce Phoenix DATE columns to TIMESTAMP through DataFrame API") { val df = spark.sqlContext.read .format("phoenix") - .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, "dateAsTimestamp" -> "true")) + .options(Map("table" -> "DATE_TEST", PhoenixDataSource.JDBC_URL -> jdbcUrl, "dateAsTimestamp" -> "true")) .load val dtRes = df.select("COL1").first() val ts = dtRes.getTimestamp(0).getTime - val epoch = new Date().getTime + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - assert(Math.abs(epoch - ts) < 300000) + assert(expected == ts) } test("Can load Phoenix Time columns through DataFrame API") { val df = spark.sqlContext.read .format("phoenix") - .options(Map("table" -> "TIME_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "TIME_TEST", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load val time = df.select("COL1").first().getTimestamp(0).getTime - val epoch = new Date().getTime - assert(Math.abs(epoch - time) < 86400000) + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli + + assert(expected == time) } test("can read all Phoenix data types") { val df = spark.sqlContext.read .format("phoenix") - .options(Map("table" -> "GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "GIGANTIC_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load df.write .format("phoenix") - .options(Map("table" -> "OUTPUT_GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map("table" -> "OUTPUT_GIGANTIC_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Overwrite) .save() @@ -938,7 +914,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { .options( Map("table" -> SchemaUtil.getEscapedArgument("TABLE_WITH_COL_FAMILY"), "doNotMapColumnFamily" -> "true", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + PhoenixDataSource.JDBC_URL -> jdbcUrl)).load val schema = df.schema @@ -955,7 +931,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { .options( Map("table" -> SchemaUtil.getEscapedArgument("TABLE_WITH_COL_FAMILY"), "doNotMapColumnFamily" -> "false", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + PhoenixDataSource.JDBC_URL -> jdbcUrl)).load val schema = df.schema diff --git a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala index 291ea2a3..166306bc 100644 --- a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala @@ -13,8 +13,8 @@ */ package org.apache.phoenix.spark -import org.apache.phoenix.util.PhoenixRuntime -import org.apache.spark.sql.SQLContext +import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource +import org.apache.spark.sql.{Row, SaveMode} import scala.collection.mutable.ListBuffer @@ -34,102 +34,141 @@ class PhoenixSparkITTenantSpecific extends AbstractPhoenixSparkIT { val OrgIdCol = "ORGANIZATION_ID" val TenantOnlyCol = "TENANT_ONLY_COL" val TenantTable = "TENANT_VIEW" + val TenantTableWithSalt = "TENANT_VIEW_WITH_SALT" // Data set for tests that write to Phoenix val TestDataSet = List(("testOrg1", "data1"), ("testOrg2", "data2"), ("testOrg3", "data3")) + val TestDataSet2 = List(("testOrg4", "data4"), ("testOrg5", "data5")) - /** - * Helper method used by write tests to verify content written. - * Assumes the caller has written the TestDataSet (defined above) to Phoenix - * and that 1 row of default data exists (upserted after table creation in tenantSetup.sql) - */ - def verifyResults(): Unit = { - // Contains the default data upserted into the tenant-specific table in tenantSetup.sql and the data upserted by tests - val VerificationDataSet = List(("defaultOrg", "defaultData")) ::: TestDataSet + val sqlTableName = "TENANT_TABLE" - val SelectStatement = "SELECT " + OrgIdCol + "," + TenantOnlyCol + " FROM " + TenantTable - val stmt = conn.createStatement() - val rs = stmt.executeQuery(SelectStatement) - - val results = ListBuffer[(String, String)]() - while (rs.next()) { - results.append((rs.getString(1), rs.getString(2))) - } - stmt.close() - results.toList shouldEqual VerificationDataSet + after { + spark.sql(s"DROP TABLE IF EXISTS $sqlTableName") } - /*****************/ - /** Read tests **/ - /*****************/ + /** ************** */ + /** Read tests * */ + /** ************** */ test("Can read from tenant-specific table as DataFrame") { - val df = spark.sqlContext.phoenixTableAsDataFrame( - TenantTable, - Seq(OrgIdCol, TenantOnlyCol), - zkUrl = Some(quorumAddress), - tenantId = Some(TenantId), - conf = hbaseConfiguration) + val expected = Array(Row.fromSeq(Seq("defaultOrg", "defaultData"))) + val df = spark.read + .format("phoenix") + .option(PhoenixDataSource.TABLE, TenantTable) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .option(PhoenixDataSource.TENANT_ID, TenantId) + .load() + .select(OrgIdCol, TenantOnlyCol) // There should only be 1 row upserted in tenantSetup.sql - val count = df.count() - count shouldEqual 1L + val result = df.collect() + expected shouldEqual result } - test("Can read from tenant-specific table as RDD") { - val rdd = spark.sparkContext.phoenixTableAsRDD( - TenantTable, - Seq(OrgIdCol, TenantOnlyCol), - zkUrl = Some(quorumAddress), - tenantId = Some(TenantId), - conf = hbaseConfiguration) + test("Can read from tenant-specific and salted table as DataFrame") { + val expected = Array(Row.fromSeq(Seq("defaultOrg", "defaultData"))) + val df = spark.read + .format("phoenix") + .option(PhoenixDataSource.TABLE, TenantTableWithSalt) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .option(PhoenixDataSource.TENANT_ID, TenantId) + .load() + .select(OrgIdCol, TenantOnlyCol) // There should only be 1 row upserted in tenantSetup.sql - val count = rdd.count() - count shouldEqual 1L + val result = df.collect() + expected shouldEqual result } - /*****************/ - /** Write tests **/ - /*****************/ + test("Can read from tenant table using spark-sql") { + + val expected = Array(Row.fromSeq(Seq("defaultOrg", "defaultData"))) + + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' '$TenantTable', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', '${PhoenixDataSource.TENANT_ID}' '${TenantId}')") + + val dataFrame = spark.sql(s"SELECT $OrgIdCol,$TenantOnlyCol FROM $sqlTableName") - test("Can write a DataFrame using 'DataFrame.saveToPhoenix' to tenant-specific view") { + dataFrame.collect() shouldEqual expected + } + + + /** ************** */ + /** Write tests * */ + /** ************** */ + + test("Can write a DataFrame using 'DataFrame.write' to tenant-specific view") { val sqlContext = spark.sqlContext import sqlContext.implicits._ val df = spark.sparkContext.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol) - df.saveToPhoenix(TenantTable, zkUrl = Some(quorumAddress), tenantId = Some(TenantId)) + df.write + .format("phoenix") + .mode(SaveMode.Overwrite) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .option(PhoenixDataSource.TABLE, TenantTable) + .option(PhoenixDataSource.TENANT_ID, TenantId) + .save() + + val expected = List(("defaultOrg", "defaultData")) ::: TestDataSet + val SelectStatement = s"SELECT $OrgIdCol,$TenantOnlyCol FROM $TenantTable" + val stmt = conn.createStatement() + val rs = stmt.executeQuery(SelectStatement) - verifyResults + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual expected } - test("Can write a DataFrame using 'DataFrame.write' to tenant-specific view") { + test("Can write a DataFrame using 'DataFrame.write' to tenant-specific with salt view") { val sqlContext = spark.sqlContext import sqlContext.implicits._ val df = spark.sparkContext.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol) - df.write .format("phoenix") - .mode("overwrite") - .option("table", TenantTable) - .option(PhoenixRuntime.TENANT_ID_ATTRIB, TenantId) - .option("zkUrl", PhoenixSparkITHelper.getUrl) + .mode(SaveMode.Overwrite) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .option(PhoenixDataSource.TABLE, TenantTableWithSalt) + .option(PhoenixDataSource.TENANT_ID, TenantId) .save() - verifyResults + val expected = List(("defaultOrg", "defaultData")) ::: TestDataSet + val SelectStatement = s"SELECT $OrgIdCol,$TenantOnlyCol FROM $TenantTableWithSalt" + val stmt = conn.createStatement() + val rs = stmt.executeQuery(SelectStatement) + + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual expected } - test("Can write an RDD to Phoenix tenant-specific view") { - spark.sparkContext - .parallelize(TestDataSet) - .saveToPhoenix( - TenantTable, - Seq(OrgIdCol, TenantOnlyCol), - hbaseConfiguration, - tenantId = Some(TenantId) - ) - - verifyResults + test("Can use write data into tenant table using spark SQL INSERT") { + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' '$TenantTable', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', '${PhoenixDataSource.TENANT_ID}' '${TenantId}')") + + (TestDataSet ::: TestDataSet2).foreach(tuple => { + // Insert data + spark.sql(s"INSERT INTO $sqlTableName VALUES('${tuple._1}', NULL, '${tuple._2}')") + }) + + val expected = List(("defaultOrg", "defaultData")) ::: TestDataSet ::: TestDataSet2 + val SelectStatement = s"SELECT $OrgIdCol,$TenantOnlyCol FROM $TenantTable" + val stmt = conn.createStatement() + val rs = stmt.executeQuery(SelectStatement) + + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual expected } + } diff --git a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala new file mode 100644 index 00000000..c6683883 --- /dev/null +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala @@ -0,0 +1,113 @@ +package org.apache.phoenix.spark + +import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource +import org.apache.spark.sql.Row + +import scala.collection.mutable.ListBuffer + +/** + * Note: If running directly from an IDE, these are the recommended VM parameters: + * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + */ +class PhoenixSparkSqlIT extends AbstractPhoenixSparkIT { + val sqlTableName = "SQL_TABLE" + + after { + spark.sql(s"DROP TABLE IF EXISTS $sqlTableName") + } + + test("Can read from table using spark-sql") { + val expected : Array[Row] = Array( + Row.fromSeq(Seq(1, "test_row_1")), + Row.fromSeq(Seq(2, "test_row_2")) + ) + + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE1', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl')") + + val dataFrame = spark.sql(s"SELECT * FROM $sqlTableName") + + dataFrame.collect() shouldEqual expected + } + + test("Can read from table using spark-sql with where clause and selecting specific columns`") { + val expected : Array[Row] = Array( + Row.fromSeq(Seq("test_row_1")) + ) + + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE1', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl')") + + val dataFrame = spark.sql(s"SELECT COL1 as LABEL FROM $sqlTableName where ID=1") + + dataFrame.collect() shouldEqual expected + } + + test("Can read from table having column family name") { + val expected : Array[Row] = Array( + Row.fromSeq(Seq(1)) + ) + + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE_WITH_COL_FAMILY', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', 'doNotMapColumnFamily' 'false')") + val dataFrame = spark.sql(s"SELECT ID FROM $sqlTableName where `DATA.COL1`='test_row_1'") + + dataFrame.collect() shouldEqual expected + } + + test("Can read from table having column family name and map column to `columnName`") { + val expected : Array[Row] = Array( + Row.fromSeq(Seq(1)) + ) + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE_WITH_COL_FAMILY', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', 'doNotMapColumnFamily' 'true')") + val dataFrame = spark.sql(s"SELECT ID FROM $sqlTableName where COL1='test_row_1'") + + dataFrame.collect() shouldEqual expected + } + + test("Can use write data using spark SQL INSERT") { + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE3', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', 'skipNormalizingIdentifier' 'true')") + + // Insert data + spark.sql(s"INSERT INTO $sqlTableName VALUES(10, 10, '10')") + spark.sql(s"INSERT INTO $sqlTableName VALUES(20, 20, '20')") + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3 WHERE ID>=10") + val expectedResults = List((10, 10, "10"), (20, 20, "20")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual expectedResults + } + + // INSERT using dataFrame as init to spark-sql + ignore("Can use write data using spark SQL INSERT using dataframe createOrReplaceTempView") { + val df1 = spark.sqlContext.read.format("phoenix") + .options( Map("table" -> "TABLE3", PhoenixDataSource.JDBC_URL -> this.jdbcUrl)).load + df1.createOrReplaceTempView("TABLE3") + + // Insert data + spark.sql("INSERT INTO TABLE3 VALUES(10, 10, 10)") + spark.sql("INSERT INTO TABLE3 VALUES(20, 20, 20)") + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3 WHERE ID>=10") + val expectedResults = List((10, 10, "10"), (20, 20, "20")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual expectedResults + } + +} diff --git a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/PhoenixDataSource.java b/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/PhoenixDataSource.java index de773d50..fc98bea3 100644 --- a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/PhoenixDataSource.java +++ b/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/PhoenixDataSource.java @@ -20,8 +20,14 @@ import java.util.Optional; import java.util.Properties; +import org.apache.phoenix.query.HBaseFactoryProvider; +import org.apache.phoenix.spark.ConfigurationUtil; import org.apache.phoenix.spark.datasource.v2.reader.PhoenixDataSourceReader; import org.apache.phoenix.spark.datasource.v2.writer.PhoenixDataSourceWriter; +import org.apache.phoenix.util.PhoenixRuntime; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.sources.BaseRelation; +import org.apache.spark.sql.sources.RelationProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.sql.SaveMode; @@ -33,6 +39,7 @@ import org.apache.spark.sql.sources.v2.reader.DataSourceReader; import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; import org.apache.spark.sql.types.StructType; +import scala.collection.immutable.Map; import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL; import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR; @@ -40,7 +47,8 @@ /** * Implements the DataSourceV2 api to read and write from Phoenix tables */ -public class PhoenixDataSource implements DataSourceV2, ReadSupport, WriteSupport, DataSourceRegister { +public class PhoenixDataSource + implements DataSourceV2, ReadSupport, WriteSupport, DataSourceRegister, RelationProvider { private static final Logger logger = LoggerFactory.getLogger(PhoenixDataSource.class); public static final String SKIP_NORMALIZING_IDENTIFIER = "skipNormalizingIdentifier"; @@ -48,6 +56,10 @@ public class PhoenixDataSource implements DataSourceV2, ReadSupport, WriteSupp public static final String ZOOKEEPER_URL = "zkUrl"; public static final String JDBC_URL = "jdbcUrl"; public static final String PHOENIX_CONFIGS = "phoenixconfigs"; + public static final String TABLE = "table"; + public static final String DATE_AS_TIME_STAMP = "dateAsTimestamp"; + public static final String DO_NOT_MAP_COLUMN_FAMILY = "doNotMapColumnFamily"; + public static final String TENANT_ID = "TenantId"; @Override public DataSourceReader createReader(DataSourceOptions options) { @@ -61,10 +73,10 @@ public Optional createWriter(String writeUUID, StructType sche } public static String getJdbcUrlFromOptions(final DataSourceOptions options) { - if (options.get(JDBC_URL).orElse(null) != null - && options.get(ZOOKEEPER_URL).orElse(null) != null) { - throw new RuntimeException("If " + JDBC_URL + " is specified, then " + ZOOKEEPER_URL - + " must not be specified"); + if (options.get(JDBC_URL).orElse(null) != null && options.get(ZOOKEEPER_URL) + .orElse(null) != null) { + throw new RuntimeException( + "If " + JDBC_URL + " is specified, then " + ZOOKEEPER_URL + " must not be specified"); } String jdbcUrl = options.get(JDBC_URL).orElse(null); @@ -88,8 +100,8 @@ public static String getJdbcUrlFromOptions(final DataSourceOptions options) { } /** - * Extract HBase and Phoenix properties that need to be set in both the driver and workers. - * We expect these properties to be passed against the key + * Extract HBase and Phoenix properties that need to be set in both the driver and workers. We + * expect these properties to be passed against the key * {@link PhoenixDataSource#PHOENIX_CONFIGS}. The corresponding value should be a * comma-separated string containing property names and property values. For example: * prop1=val1,prop2=val2,prop3=val3 @@ -108,8 +120,8 @@ public static Properties extractPhoenixHBaseConfFromOptions(final DataSourceOpti try { confToSet.setProperty(confKeyVal[0], confKeyVal[1]); } catch (ArrayIndexOutOfBoundsException e) { - throw new RuntimeException("Incorrect format for phoenix/HBase configs. " - + "Expected format: =,=,=..", + throw new RuntimeException( + "Incorrect format for phoenix/HBase configs. " + "Expected format: =,=,=..", e); } } @@ -117,6 +129,10 @@ public static Properties extractPhoenixHBaseConfFromOptions(final DataSourceOpti if (logger.isDebugEnabled()) { logger.debug("Got the following Phoenix/HBase config:\n" + confToSet); } + String tenantId = options.get(TENANT_ID).orElse(null); + if (tenantId != null) { + confToSet.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId); + } } return confToSet; } @@ -125,4 +141,10 @@ public static Properties extractPhoenixHBaseConfFromOptions(final DataSourceOpti public String shortName() { return "phoenix"; } + + @Override + public BaseRelation createRelation(SQLContext sqlContext, Map parameters) { + return new PhoenixSparkSqlRelation(sqlContext.sparkSession(), parameters); + } + } diff --git a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReader.java b/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReader.java index 22e0cfbe..3638b278 100644 --- a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReader.java +++ b/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReader.java @@ -23,13 +23,14 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.mapreduce.RegionSizeCalculator; import org.apache.phoenix.compile.QueryPlan; -import org.apache.phoenix.coprocessor.generated.PTableProtos.PTable; import org.apache.phoenix.iterate.MapReduceParallelScanGrouper; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixStatement; import org.apache.phoenix.mapreduce.PhoenixInputSplit; import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil; import org.apache.phoenix.query.KeyRange; +import org.apache.phoenix.schema.PColumn; +import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableImpl; import org.apache.phoenix.spark.FilterExpressionCompiler; import org.apache.phoenix.spark.SparkSchemaUtil; @@ -37,6 +38,7 @@ import org.apache.phoenix.util.ColumnInfo; import org.apache.phoenix.util.PhoenixRuntime; import org.apache.phoenix.util.QueryUtil; +import org.apache.phoenix.util.SchemaUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.Filter; import org.apache.spark.sql.sources.v2.DataSourceOptions; @@ -83,8 +85,8 @@ public PhoenixDataSourceReader(DataSourceOptions options) { this.options = options; this.tableName = options.tableName().get(); this.jdbcUrl = PhoenixDataSource.getJdbcUrlFromOptions(options); - this.dateAsTimestamp = options.getBoolean("dateAsTimestamp", false); - this.doNotMapColumnFamily = options.getBoolean("doNotMapColumnFamily", false); + this.dateAsTimestamp = options.getBoolean(PhoenixDataSource.DATE_AS_TIME_STAMP, false); + this.doNotMapColumnFamily = options.getBoolean(PhoenixDataSource.DO_NOT_MAP_COLUMN_FAMILY, false); this.overriddenProps = PhoenixDataSource.extractPhoenixHBaseConfFromOptions(options); setSchema(); } @@ -94,11 +96,10 @@ public PhoenixDataSourceReader(DataSourceOptions options) { */ private void setSchema() { try (Connection conn = DriverManager.getConnection(jdbcUrl, overriddenProps)) { - List columnInfos = PhoenixRuntime.generateColumnInfo(conn, tableName, null); + List columnInfos = generateColumnInfo(conn, tableName); Seq columnInfoSeq = JavaConverters.asScalaIteratorConverter(columnInfos.iterator()).asScala().toSeq(); schema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoSeq, dateAsTimestamp, doNotMapColumnFamily); - } - catch (SQLException e) { + } catch (SQLException e) { throw new RuntimeException(e); } } @@ -124,22 +125,22 @@ public Filter[] pushFilters(Filter[] filters) { @Override public List> planInputPartitions() { Optional currentScnValue = options.get(PhoenixConfigurationUtil.CURRENT_SCN_VALUE); - Optional tenantId = options.get(PhoenixConfigurationUtil.MAPREDUCE_TENANT_ID); + Optional tenantId = options.get(PhoenixDataSource.TENANT_ID); // Generate splits based off statistics, or just region splits? boolean splitByStats = options.getBoolean( PhoenixConfigurationUtil.MAPREDUCE_SPLIT_BY_STATS, PhoenixConfigurationUtil.DEFAULT_SPLIT_BY_STATS); if (currentScnValue.isPresent()) { overriddenProps.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, currentScnValue.get()); } - if (tenantId.isPresent()){ + if (tenantId.isPresent()) { overriddenProps.put(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId.get()); } try (Connection conn = DriverManager.getConnection(jdbcUrl, overriddenProps)) { List columnInfos = PhoenixRuntime.generateColumnInfo(conn, tableName, new ArrayList<>( - Arrays.asList(schema.names()))); + Arrays.asList(schema.names()))); final Statement statement = conn.createStatement(); final String selectStatement = QueryUtil.constructSelectStatement(tableName, columnInfos, whereClause); - if (selectStatement == null){ + if (selectStatement == null) { throw new NullPointerException(); } @@ -177,13 +178,13 @@ public List> planInputPartitions() { // Get the region size RegionSizeCalculator sizeCalculator = new RegionSizeCalculator(regionLocator, connection.getAdmin()); - long regionSize = sizeCalculator.getRegionSize(location.getRegionInfo().getRegionName()); + long regionSize = sizeCalculator.getRegionSize(location.getRegionInfo().getRegionName()); byte[] pTableCacheBytes = PTableImpl.toProto(queryPlan.getTableRef().getTable()). - toByteArray(); + toByteArray(); PhoenixDataSourceReadOptions phoenixDataSourceOptions = - new PhoenixDataSourceReadOptions(jdbcUrl, currentScnValue.orElse(null), - tenantId.orElse(null), selectStatement, overriddenProps, - pTableCacheBytes); + new PhoenixDataSourceReadOptions(jdbcUrl, currentScnValue.orElse(null), + tenantId.orElse(null), selectStatement, overriddenProps, + pTableCacheBytes); if (splitByStats) { for (Scan aScan : scans) { partitions.add(getInputPartition(phoenixDataSourceOptions, @@ -207,6 +208,27 @@ public Filter[] pushedFilters() { @Override public void pruneColumns(StructType schema) { - this.schema = schema; + if (schema.fields() != null && schema.fields().length != 0) + this.schema = schema; + } + + //TODO Method PhoenixRuntime.generateColumnInfo skip only salt column, add skip tenant_id column. + private List generateColumnInfo(Connection conn, String tableName) throws SQLException { + List columnInfos = new ArrayList<>(); + PTable table = PhoenixRuntime.getTable(conn, SchemaUtil.normalizeFullTableName(tableName)); + int startOffset = 0; + + if(table.getTenantId()!=null) { + startOffset++; + } + if(table.getBucketNum()!=null){ + startOffset++; + } + + for (int offset = startOffset; offset < table.getColumns().size(); offset++) { + PColumn column = table.getColumns().get(offset); + columnInfos.add(PhoenixRuntime.getColumnInfo(column)); + } + return columnInfos; } } diff --git a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartitionReader.java b/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartitionReader.java index 3a4ef656..36250855 100644 --- a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartitionReader.java +++ b/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartitionReader.java @@ -32,7 +32,6 @@ import org.apache.phoenix.compile.QueryPlan; import org.apache.phoenix.compile.StatementContext; -import org.apache.phoenix.coprocessor.generated.PTableProtos; import org.apache.phoenix.coprocessor.generated.PTableProtos.PTable; import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants; import org.apache.phoenix.iterate.ConcatResultIterator; diff --git a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataSourceWriter.java b/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataSourceWriter.java index 08fe8862..fb159f5f 100644 --- a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataSourceWriter.java +++ b/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataSourceWriter.java @@ -18,7 +18,6 @@ package org.apache.phoenix.spark.datasource.v2.writer; import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource; -import org.apache.phoenix.util.PhoenixRuntime; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.DataSourceOptions; @@ -69,7 +68,7 @@ PhoenixDataSourceWriteOptions getOptions() { private PhoenixDataSourceWriteOptions createPhoenixDataSourceWriteOptions(DataSourceOptions options, StructType schema) { String scn = options.get(CURRENT_SCN_VALUE).orElse(null); - String tenantId = options.get(PhoenixRuntime.TENANT_ID_ATTRIB).orElse(null); + String tenantId = options.get(PhoenixDataSource.TENANT_ID).orElse(null); String jdbcUrl = PhoenixDataSource.getJdbcUrlFromOptions(options); boolean skipNormalizingIdentifier = options.getBoolean(SKIP_NORMALIZING_IDENTIFIER, false); return new PhoenixDataSourceWriteOptions.Builder() diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala index 111f021c..14ad8420 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala @@ -17,14 +17,12 @@ */ package org.apache.phoenix.spark -import java.sql.Date -import java.sql.Timestamp -import java.text.Format - -import org.apache.phoenix.util.{DateUtil, SchemaUtil} import org.apache.phoenix.util.StringUtil.escapeStringConstant -import org.apache.spark.sql.sources.{And, EqualTo, Filter, GreaterThan, GreaterThanOrEqual, In, -IsNotNull, IsNull, LessThan, LessThanOrEqual, Not, Or, StringContains, StringEndsWith, StringStartsWith} +import org.apache.phoenix.util.{DateUtil, SchemaUtil} +import org.apache.spark.sql.sources._ + +import java.sql.{Date, Timestamp} +import java.text.Format class FilterExpressionCompiler() { diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala index 89d808de..c0af5c4a 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala @@ -14,7 +14,6 @@ package org.apache.phoenix.spark import java.sql.DriverManager - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hbase.HConstants import org.apache.hadoop.io.NullWritable diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala index aacd4600..2edf5463 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala @@ -19,7 +19,7 @@ package org.apache.phoenix.spark import org.apache.hadoop.conf.Configuration import org.apache.spark.rdd.RDD -import org.apache.spark.sql.sources.{BaseRelation, PrunedFilteredScan, Filter} +import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Row, SQLContext} diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala index 19535bfc..dd5dbefd 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala @@ -18,18 +18,9 @@ package org.apache.phoenix.spark import org.apache.phoenix.query.QueryConstants -import org.apache.phoenix.schema.types.{PBinary, PBinaryArray, PBoolean, PBooleanArray, PChar, -PCharArray, PDate, PDateArray, PDecimal, PDecimalArray, PDouble, PDoubleArray, PFloat, PFloatArray, -PInteger, PIntegerArray, PLong, PLongArray, PSmallint, PSmallintArray, PTime, PTimeArray, -PTimestamp, PTimestampArray, PTinyint, PTinyintArray, PUnsignedDate, PUnsignedDateArray, -PUnsignedDouble, PUnsignedDoubleArray, PUnsignedFloat, PUnsignedFloatArray, PUnsignedInt, -PUnsignedIntArray, PUnsignedLong, PUnsignedLongArray, PUnsignedSmallint, PUnsignedSmallintArray, -PUnsignedTime, PUnsignedTimeArray, PUnsignedTimestamp, PUnsignedTimestampArray, PUnsignedTinyint, -PUnsignedTinyintArray, PVarbinary, PVarbinaryArray, PVarchar, PVarcharArray} +import org.apache.phoenix.schema.types._ import org.apache.phoenix.util.{ColumnInfo, SchemaUtil} -import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteType, DataType, DateType, -DecimalType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructField, -StructType, TimestampType} +import org.apache.spark.sql.types._ object SparkSchemaUtil { diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/PhoenixSparkSqlRelation.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/PhoenixSparkSqlRelation.scala new file mode 100644 index 00000000..5143b1cc --- /dev/null +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/PhoenixSparkSqlRelation.scala @@ -0,0 +1,58 @@ +package org.apache.phoenix.spark.datasource.v2 + +import org.apache.phoenix.spark.datasource.v2.reader.PhoenixDataSourceReader +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode, SparkSession} +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.execution.datasources.v2.DataSourceRDD +import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation, PrunedFilteredScan} +import org.apache.spark.sql.types.StructType + +import scala.collection.JavaConverters._ + +case class PhoenixSparkSqlRelation( + @transient sparkSession: SparkSession, + params: Map[String, String] + ) extends BaseRelation with PrunedFilteredScan with InsertableRelation { + + override def schema: StructType = dataSourceReader.readSchema() + + override def sqlContext: SQLContext = sparkSession.sqlContext + + private def dataSourceReader: PhoenixDataSourceReader = new PhoenixDataSourceReader(dataSourceOptions) + + private def dataSourceOptions = new DataSourceOptions(params.asJava) + + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { + val requiredSchema = StructType(requiredColumns.flatMap(c => schema.fields.find(_.name == c))) + + val reader: PhoenixDataSourceReader = dataSourceReader + reader.pushFilters(filters) + reader.pruneColumns(requiredSchema) + val rdd = new DataSourceRDD( + sqlContext.sparkContext, + reader.planInputPartitions().asScala + ) + rdd.map(ir => { + new GenericRowWithSchema(ir.toSeq(requiredSchema).toArray, requiredSchema) + }) + } + + + override def insert(data: DataFrame, overwrite: Boolean): Unit = { + data + .write + .format("phoenix") + .option(PhoenixDataSource.TABLE, params(PhoenixDataSource.TABLE)) + .option(PhoenixDataSource.JDBC_URL, PhoenixDataSource.getJdbcUrlFromOptions(dataSourceOptions)) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER, + params.getOrElse(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER, "false")) + .option(PhoenixDataSource.TENANT_ID,params.getOrElse(PhoenixDataSource.TENANT_ID,null)) + .mode(SaveMode.Overwrite) + .save() + } + + +} + diff --git a/phoenix5-spark3/README.md b/phoenix5-spark3/README.md index 8954e168..a3f03271 100644 --- a/phoenix5-spark3/README.md +++ b/phoenix5-spark3/README.md @@ -28,6 +28,19 @@ Apart from the shaded connector JAR, you also need to add the hbase mapredcp lib (add the exact paths as appropiate to your system) Both the `spark.driver.extraClassPath` and `spark.executor.extraClassPath` properties need to be set the above classpath. You may add them spark-defaults.conf, or specify them on the spark-shell or spark-submit command line. +## Configuration properties + +| Name | Default | Usage | Description | +|---------------------------|---------|-------|-----------------------------------------------------------------------------------------------------------------------------------------------| +| table | empty | R/W | table name as `namespace.table_name` | +| zkUrl | empty | R/W | (Optional) List of zookeeper hosts. Deprecated, use `jdbcUrl` instead. Recommended not to set, value will be taken from hbase-site.xml | +| jdbcUrl | empty | R/W | (Optional) jdbc url connection to database as `jdbc:phoenix:zkHost:zkport`. Recommended not to set, value will be taken from hbase-site.xml | +| dateAsTimestamp | false | R | Cast Date to Timestamp | +| doNotMapColumnFamily | false | R | For non default column family. Do not prefix column with column family name | +| TenantId | empty | R/W | Define tenantId when reading from multitenant table | +| phoenixconfigs | empty | R/W | Comma seperated value of hbase/phoenix config to override. (property=value,property=value) | +| skipNormalizingIdentifier | empty | W | skip normalize identifier | + ## Reading Phoenix Tables Given a Phoenix table with the following DDL and DML: @@ -55,12 +68,14 @@ val spark = SparkSession val df = spark.sqlContext .read .format("phoenix") - .options(Map("table" -> "TABLE1")) + .option("table", "TABLE1") .load df.filter(df("COL1") === "test_row_1" && df("ID") === 1L) .select(df("ID")) .show + +spark.stop() ``` Java example: ```java @@ -75,21 +90,68 @@ public class PhoenixSparkRead { public static void main() throws Exception { SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test") .set("spark.hadoopRDD.ignoreEmptySplits", "false"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSessinon spark = SparkSession.builder().config(sparkConf).getOrCreate(); // Load data from TABLE1 - Dataset df = sqlContext + Dataset df = spark .read() .format("phoenix") .option("table", "TABLE1") .load(); df.createOrReplaceTempView("TABLE1"); - SQLContext sqlCtx = new SQLContext(jsc); - df = sqlCtx.sql("SELECT * FROM TABLE1 WHERE COL1='test_row_1' AND ID=1L"); + df = spark.sql("SELECT * FROM TABLE1 WHERE COL1='test_row_1' AND ID=1L"); df.show(); - jsc.stop(); + + spark.stop(); + } +} +``` + +### Load as a DataFrame using SparkSql and the DataSourceV2 API +Scala example: +```scala +import org.apache.spark.SparkContext +import org.apache.spark.sql.{SQLContext, SparkSession} + +val spark = SparkSession + .builder() + .appName("phoenix-test") + .master("local") + .config("spark.hadoopRDD.ignoreEmptySplits", "false") + .getOrCreate() + +// Load data from TABLE1 +spark.sql("CREATE TABLE TABLE1_SQL USING phoenix OPTIONS ('table' 'TABLE1')") + +val df = spark.sql(s"SELECT ID FROM TABLE1_SQL where COL1='test_row_1'") + +df.show + +spark.stop() +``` +Java example: +```java +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; + +public class PhoenixSparkRead { + + public static void main() throws Exception { + SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test") + .set("spark.hadoopRDD.ignoreEmptySplits", "false"); + SparkSessinon spark = SparkSession.builder().config(sparkConf).getOrCreate(); + + // Load data from TABLE1 + Dataset df = spark.sql("CREATE TABLE TABLE1_SQL USING phoenix OPTIONS ('table' 'TABLE1'"); + + df = spark.sql("SELECT * FROM TABLE1_SQL WHERE COL1='test_row_1' AND ID=1L"); + df.show(); + + spark.stop(); } } ``` @@ -99,8 +161,8 @@ public class PhoenixSparkRead { ### Save DataFrames to Phoenix using DataSourceV2 The `save` is method on DataFrame allows passing in a data source type. You can use -`phoenix` for DataSourceV2 and must also pass in a `table` and `zkUrl` parameter to -specify which table and server to persist the DataFrame to. The column names are derived from +`phoenix` for DataSourceV2 and must also pass in a `table` parameter to +specify which table to persist the DataFrame to. The column names are derived from the DataFrame's schema field names, and must match the Phoenix column names. The `save` method also takes a `SaveMode` option, for which only `SaveMode.Append` is supported. @@ -128,15 +190,16 @@ val spark = SparkSession val df = spark.sqlContext .read .format("phoenix") - .options(Map("table" -> "INPUT_TABLE")) + .option("table", "INPUT_TABLE") .load // Save to OUTPUT_TABLE df.write .format("phoenix") .mode(SaveMode.Append) - .options(Map("table" -> "OUTPUT_TABLE")) + .option("table", "OUTPUT_TABLE") .save() +spark.stop() ``` Java example: ```java @@ -152,11 +215,10 @@ public class PhoenixSparkWriteFromInputTable { public static void main() throws Exception { SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test") .set("spark.hadoopRDD.ignoreEmptySplits", "false"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSessinon spark = SparkSession.builder().config(sparkConf).getOrCreate(); // Load INPUT_TABLE - Dataset df = sqlContext + Dataset df = spark .read() .format("phoenix") .option("table", "INPUT_TABLE") @@ -168,15 +230,16 @@ public class PhoenixSparkWriteFromInputTable { .mode(SaveMode.Append) .option("table", "OUTPUT_TABLE") .save(); - jsc.stop(); + + spark.stop(); } } ``` ### Save from an external RDD with a schema to a Phoenix table -Just like the previous example, you can pass in the data source type as `phoenix` and specify the `table` and -`zkUrl` parameters indicating which table and server to persist the DataFrame to. +Just like the previous example, you can pass in the data source type as `phoenix` and specify the `table` parameter +indicating which table to persist the DataFrame to. Note that the schema of the RDD must match its column data and this must match the schema of the Phoenix table that you save to. @@ -210,13 +273,15 @@ val schema = StructType( val rowRDD = spark.sparkContext.parallelize(dataSet) // Apply the schema to the RDD. -val df = spark.sqlContext.createDataFrame(rowRDD, schema) +val df = spark.createDataFrame(rowRDD, schema) df.write .format("phoenix") - .options(Map("table" -> "OUTPUT_TABLE")) + .option("table", "OUTPUT_TABLE") .mode(SaveMode.Append) .save() + +spark.stop() ``` Java example: ```java @@ -240,10 +305,7 @@ public class PhoenixSparkWriteFromRDDWithSchema { public static void main() throws Exception { SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test") .set("spark.hadoopRDD.ignoreEmptySplits", "false"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(jsc); - SparkSession spark = sqlContext.sparkSession(); - Dataset df; + SparkSessinon spark = SparkSession.builder().config(sparkConf).getOrCreate(); // Generate the schema based on the fields List fields = new ArrayList<>(); @@ -259,14 +321,14 @@ public class PhoenixSparkWriteFromRDDWithSchema { } // Create a DataFrame from the rows and the specified schema - df = spark.createDataFrame(rows, schema); + Dataset df = spark.createDataFrame(rows, schema); df.write() .format("phoenix") .mode(SaveMode.Append) .option("table", "OUTPUT_TABLE") .save(); - - jsc.stop(); + + spark.stop(); } } ``` @@ -280,13 +342,13 @@ it falls back to using connection defined by hbase-site.xml. - `"jdbcUrl"` expects a full Phoenix JDBC URL, i.e. "jdbc:phoenix" or "jdbc:phoenix:zkHost:zkport", while `"zkUrl"` expects the ZK quorum only, i.e. "zkHost:zkPort" - If you want to use DataSourceV1, you can use source type `"org.apache.phoenix.spark"` -instead of `"phoenix"`, however this is deprecated as of `connectors-1.0.0`. -The `"org.apache.phoenix.spark"` datasource does not accept the `"jdbcUrl"` parameter, -only `"zkUrl"` + instead of `"phoenix"`, however this is deprecated. + The `"org.apache.phoenix.spark"` datasource does not accept the `"jdbcUrl"` parameter, + only `"zkUrl"` - The (deprecated) functions `phoenixTableAsDataFrame`, `phoenixTableAsRDD` and -`saveToPhoenix` use the deprecated `"org.apache.phoenix.spark"` datasource, and allow -optionally specifying a `conf` Hadoop configuration parameter with custom Phoenix client settings, -as well as an optional `zkUrl` parameter. + `saveToPhoenix` use the deprecated `"org.apache.phoenix.spark"` datasource, and allow + optionally specifying a `conf` Hadoop configuration parameter with custom Phoenix client settings, + as well as an optional `zkUrl` parameter. - As of [PHOENIX-5197](https://issues.apache.org/jira/browse/PHOENIX-5197), you can pass configurations from the driver to executors as a comma-separated list against the key `phoenixConfigs` i.e (PhoenixDataSource.PHOENIX_CONFIGS), for ex: @@ -295,7 +357,7 @@ to executors as a comma-separated list against the key `phoenixConfigs` i.e (Pho .sqlContext .read .format("phoenix") - .options(Map("table" -> "Table1", "jdbcUrl" -> "jdbc:phoenix:phoenix-server:2181", "phoenixConfigs" -> "hbase.client.retries.number=10,hbase.client.pause=10000")) + .options(Map("table" -> "Table1", "phoenixConfigs" -> "hbase.client.retries.number=10,hbase.client.pause=10000")) .load; ``` This list of properties is parsed and populated into a properties map which is passed to `DriverManager.getConnection(connString, propsMap)`. @@ -309,9 +371,10 @@ to executors as a comma-separated list against the key `phoenixConfigs` i.e (Pho .sqlContext .read .format("phoenix") - .options(Map("table" -> "Table1", "jdbcUrl" -> "jdbc:phoenix:phoenix-server:2181", "doNotMapColumnFamily" -> "true")) + .options(Map("table" -> "Table1", "doNotMapColumnFamily" -> "true")) .load; ``` + ## Limitations - Basic support for column and predicate pushdown using the Data Source API @@ -332,7 +395,7 @@ import org.apache.phoenix.spark._ val configuration = new Configuration() // Can set Phoenix-specific settings, requires 'hbase.zookeeper.quorum' -val sparkConf = new SparkConf().set("spark.ui.showConsoleProgress", "false") +val sparkConf = new SparkConf() val sc = new SparkContext("local", "phoenix-test", sparkConf) val sqlContext = new SQLContext(sc) @@ -381,7 +444,7 @@ import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.phoenix.spark._ -val sparkConf = new SparkConf().set("spark.ui.showConsoleProgress", "false") +val sparkConf = new SparkConf() val sc = new SparkContext("local", "phoenix-test", sparkConf) val dataSet = List((1L, "1", 1), (2L, "2", 2), (3L, "3", 3)) @@ -392,4 +455,4 @@ sc Seq("ID","COL1","COL2"), zkUrl = Some("phoenix-server:2181") ) -``` \ No newline at end of file +``` diff --git a/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/DataSourceApiIT.java b/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/DataSourceApiIT.java index efcef710..64fb0f5b 100644 --- a/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/DataSourceApiIT.java +++ b/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/DataSourceApiIT.java @@ -31,7 +31,6 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -99,7 +98,7 @@ public void basicWriteAndReadBackTest() throws SQLException { schema); df1.write().format("phoenix").mode(SaveMode.Append) - .option("table", tableName) + .option(PhoenixDataSource.TABLE, tableName) .option(ZOOKEEPER_URL, getUrl()) .save(); @@ -117,7 +116,7 @@ public void basicWriteAndReadBackTest() throws SQLException { schema); df2.write().format("phoenix").mode(SaveMode.Append) - .option("table", tableName) + .option(PhoenixDataSource.TABLE, tableName) .option(JDBC_URL, jdbcUrl) .save(); @@ -128,7 +127,7 @@ public void basicWriteAndReadBackTest() throws SQLException { schema); df3.write().format("phoenix").mode(SaveMode.Append) - .option("table", tableName) + .option(PhoenixDataSource.TABLE, tableName) .save(); try (Connection conn = DriverManager.getConnection(getUrl()); @@ -147,14 +146,14 @@ public void basicWriteAndReadBackTest() throws SQLException { } Dataset df1Read = spark.read().format("phoenix") - .option("table", tableName) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.TABLE, tableName) + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); assertEquals(3l, df1Read.count()); // Use jdbcUrl Dataset df2Read = spark.read().format("phoenix") - .option("table", tableName) + .option(PhoenixDataSource.TABLE, tableName) .option(PhoenixDataSource.JDBC_URL, jdbcUrl) .load(); @@ -162,7 +161,7 @@ public void basicWriteAndReadBackTest() throws SQLException { // Use default Dataset df3Read = spark.read().format("phoenix") - .option("table", tableName) + .option(PhoenixDataSource.TABLE, tableName) .load(); assertEquals(3l, df3Read.count()); @@ -173,7 +172,6 @@ public void basicWriteAndReadBackTest() throws SQLException { } @Test - @Ignore // Spark3 seems to be unable to handle mixed case colum names public void lowerCaseWriteTest() throws SQLException { SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test"); JavaSparkContext jsc = new JavaSparkContext(sparkConf); @@ -203,8 +201,9 @@ public void lowerCaseWriteTest() throws SQLException { df.write() .format("phoenix") .mode(SaveMode.Append) - .option("table", tableName) - .option(ZOOKEEPER_URL, getUrl()) + .option(PhoenixDataSource.TABLE, tableName) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER,"true") + .option(JDBC_URL, getUrl()) .save(); try (Connection conn = DriverManager.getConnection(getUrl()); diff --git a/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/OrderByIT.java b/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/OrderByIT.java index ee87f459..e278f4de 100644 --- a/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/OrderByIT.java +++ b/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/OrderByIT.java @@ -43,15 +43,9 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.Ignore; - - - -import scala.Option; -import scala.collection.JavaConverters; @Category(ParallelStatsDisabledTest.class) public class OrderByIT extends BaseOrderByIT { @@ -134,12 +128,12 @@ public void testOrderByWithJoin() throws Exception { Arrays.asList("A_STRING", "CF1.A", "CF1.B", "COL1", "CF2.C", "CF2.D")); SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") - .option("table", tableName1) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.TABLE, tableName1) + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName1); phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") - .option("table", tableName2) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.TABLE, tableName2) + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName2); String query = @@ -250,12 +244,12 @@ public void testOrderByWithUnionAll() throws Exception { SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") - .option("table", tableName1) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.TABLE, tableName1) + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName1); phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") - .option("table", tableName2) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.TABLE, tableName2) + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName2); String query = @@ -300,8 +294,8 @@ public void testCombinationOfOrAndFilters() throws Exception { createTestTable(getUrl(), ddl); SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") - .option("table", tableName1) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.TABLE, tableName1) + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName1); String dml = "UPSERT INTO " + tableName1 + " VALUES(?,?,?,?,?)"; PreparedStatement stmt = conn.prepareStatement(dml); @@ -391,8 +385,8 @@ public void testOrderByWithExpression() throws Exception { SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") - .option("table", tableName) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.TABLE, tableName) + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName); Dataset dataset = sqlContext.sql("SELECT col1+col2, col4, a_string FROM " + tableName @@ -453,8 +447,8 @@ public void testColumnFamily() throws Exception { SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext(); Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix") - .option("table", tableName) - .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load(); + .option(PhoenixDataSource.TABLE, tableName) + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); phoenixDataSet.createOrReplaceTempView(tableName); Dataset dataset = sqlContext.sql("SELECT A_STRING, `CF1.A`, `CF1.B`, COL1, `CF2.C`, `CF2.D`, COL2 from " diff --git a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/SparkResultSet.java b/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/SparkResultSet.java similarity index 100% rename from phoenix5-spark3/src/main/java/org/apache/phoenix/spark/SparkResultSet.java rename to phoenix5-spark3/src/it/java/org/apache/phoenix/spark/SparkResultSet.java diff --git a/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/SparkUtil.java b/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/SparkUtil.java index 674e02ff..8c7dacc9 100644 --- a/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/SparkUtil.java +++ b/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/SparkUtil.java @@ -28,9 +28,6 @@ import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.execution.SparkPlan; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import scala.Option; -import scala.collection.JavaConverters; import java.sql.Connection; import java.sql.ResultSet; @@ -65,8 +62,8 @@ public static ResultSet executeQuery(Connection conn, QueryBuilder queryBuilder, // create PhoenixRDD using the table name and columns that are required by the query // since we don't set the predicate filtering is done after rows are returned from spark Dataset phoenixDataSet = getSparkSession().read().format("phoenix") - .option("table", queryBuilder.getFullTableName()) - .option(PhoenixDataSource.ZOOKEEPER_URL, url).load(); + .option(PhoenixDataSource.TABLE, queryBuilder.getFullTableName()) + .option(PhoenixDataSource.JDBC_URL, url).load(); phoenixDataSet.createOrReplaceTempView(queryBuilder.getFullTableName()); Dataset dataset = sqlContext.sql(queryBuilder.build()); diff --git a/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/sql/connector/PhoenixTestingDataSource.java b/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/sql/connector/PhoenixTestingDataSource.java index adfbe73a..c3e1593a 100644 --- a/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/sql/connector/PhoenixTestingDataSource.java +++ b/phoenix5-spark3/src/it/java/org/apache/phoenix/spark/sql/connector/PhoenixTestingDataSource.java @@ -41,19 +41,18 @@ public class PhoenixTestingDataSource extends PhoenixDataSource { @Override public StructType inferSchema(CaseInsensitiveStringMap options) { - String tableName = options.get("table"); + String tableName = options.get(PhoenixDataSource.TABLE); String jdbcUrl = getJdbcUrlFromOptions(options); - boolean dateAsTimestamp = Boolean.parseBoolean(options.getOrDefault("dateAsTimestamp", Boolean.toString(false))); + boolean dateAsTimestamp = Boolean.parseBoolean(options.getOrDefault(PhoenixDataSource.DATE_AS_TIME_STAMP, Boolean.toString(false))); Properties overriddenProps = extractPhoenixHBaseConfFromOptions(options); try (Connection conn = DriverManager.getConnection(jdbcUrl, overriddenProps)) { List columnInfos = PhoenixRuntime.generateColumnInfo(conn, tableName, null); Seq columnInfoSeq = JavaConverters.asScalaIteratorConverter(columnInfos.iterator()).asScala().toSeq(); - schema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoSeq, dateAsTimestamp, false); + return SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoSeq, dateAsTimestamp, false); } catch (SQLException e) { throw new RuntimeException(e); } - return schema; } @Override diff --git a/phoenix5-spark3/src/it/resources/globalSetup.sql b/phoenix5-spark3/src/it/resources/globalSetup.sql index 6082727e..32a74554 100644 --- a/phoenix5-spark3/src/it/resources/globalSetup.sql +++ b/phoenix5-spark3/src/it/resources/globalSetup.sql @@ -19,9 +19,9 @@ CREATE TABLE table1_copy (id BIGINT NOT NULL PRIMARY KEY, col1 VARCHAR) CREATE TABLE table2 (id BIGINT NOT NULL PRIMARY KEY, table1_id BIGINT, "t2col1" VARCHAR) CREATE TABLE table3 (id BIGINT NOT NULL PRIMARY KEY, table3_id BIGINT, "t2col1" VARCHAR) UPSERT INTO table1 (id, col1) VALUES (1, 'test_row_1') +UPSERT INTO table1 (id, col1) VALUES (2, 'test_row_2') UPSERT INTO table2 (id, table1_id, "t2col1") VALUES (1, 1, 'test_child_1') UPSERT INTO table2 (id, table1_id, "t2col1") VALUES (2, 1, 'test_child_2') -UPSERT INTO table1 (id, col1) VALUES (2, 'test_row_2') UPSERT INTO table2 (id, table1_id, "t2col1") VALUES (3, 2, 'test_child_1') UPSERT INTO table2 (id, table1_id, "t2col1") VALUES (4, 2, 'test_child_2') UPSERT INTO table2 (id, table1_id, "t2col1") VALUES (5, 2, 'test_child_3') @@ -50,9 +50,9 @@ UPSERT INTO TEST_DECIMAL VALUES (1, 123.456789) CREATE TABLE TEST_SMALL_TINY (ID BIGINT NOT NULL PRIMARY KEY, COL1 SMALLINT, COL2 TINYINT) UPSERT INTO TEST_SMALL_TINY VALUES (1, 32767, 127) CREATE TABLE DATE_TEST(ID BIGINT NOT NULL PRIMARY KEY, COL1 DATE) -UPSERT INTO DATE_TEST VALUES(1, CURRENT_DATE()) +UPSERT INTO DATE_TEST VALUES(1, '2021-01-01T00:00:00Z') CREATE TABLE TIME_TEST(ID BIGINT NOT NULL PRIMARY KEY, COL1 TIME) -UPSERT INTO TIME_TEST VALUES(1, CURRENT_TIME()) +UPSERT INTO TIME_TEST VALUES(1, '2021-01-01T00:00:00Z') CREATE TABLE "space" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR) UPSERT INTO "space" VALUES ('key1', 'xyz') CREATE TABLE "small" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR, "salary" INTEGER ) @@ -61,9 +61,11 @@ UPSERT INTO "small" VALUES ('key2', 'bar', 20000) UPSERT INTO "small" VALUES ('key3', 'xyz', 30000) CREATE TABLE MULTITENANT_TEST_TABLE (TENANT_ID VARCHAR NOT NULL, ORGANIZATION_ID VARCHAR, GLOBAL_COL1 VARCHAR CONSTRAINT pk PRIMARY KEY (TENANT_ID, ORGANIZATION_ID)) MULTI_TENANT=true +CREATE TABLE MULTITENANT_TEST_TABLE_WITH_SALT (TENANT_ID VARCHAR NOT NULL, ORGANIZATION_ID VARCHAR, GLOBAL_COL1 VARCHAR CONSTRAINT pk PRIMARY KEY (TENANT_ID, ORGANIZATION_ID)) MULTI_TENANT=true, SALT_BUCKETS = 20 CREATE TABLE IF NOT EXISTS GIGANTIC_TABLE (ID INTEGER PRIMARY KEY,unsig_id UNSIGNED_INT,big_id BIGINT,unsig_long_id UNSIGNED_LONG,tiny_id TINYINT,unsig_tiny_id UNSIGNED_TINYINT,small_id SMALLINT,unsig_small_id UNSIGNED_SMALLINT,float_id FLOAT,unsig_float_id UNSIGNED_FLOAT,double_id DOUBLE,unsig_double_id UNSIGNED_DOUBLE,decimal_id DECIMAL,boolean_id BOOLEAN,time_id TIME,date_id DATE,timestamp_id TIMESTAMP,unsig_time_id UNSIGNED_TIME,unsig_date_id UNSIGNED_DATE,unsig_timestamp_id UNSIGNED_TIMESTAMP,varchar_id VARCHAR (30),char_id CHAR (30),binary_id BINARY (100),varbinary_id VARBINARY (100)) CREATE TABLE IF NOT EXISTS OUTPUT_GIGANTIC_TABLE (ID INTEGER PRIMARY KEY,unsig_id UNSIGNED_INT,big_id BIGINT,unsig_long_id UNSIGNED_LONG,tiny_id TINYINT,unsig_tiny_id UNSIGNED_TINYINT,small_id SMALLINT,unsig_small_id UNSIGNED_SMALLINT,float_id FLOAT,unsig_float_id UNSIGNED_FLOAT,double_id DOUBLE,unsig_double_id UNSIGNED_DOUBLE,decimal_id DECIMAL,boolean_id BOOLEAN,time_id TIME,date_id DATE,timestamp_id TIMESTAMP,unsig_time_id UNSIGNED_TIME,unsig_date_id UNSIGNED_DATE,unsig_timestamp_id UNSIGNED_TIMESTAMP,varchar_id VARCHAR (30),char_id CHAR (30),binary_id BINARY (100),varbinary_id VARBINARY (100)) UPSERT INTO GIGANTIC_TABLE VALUES(0,2,3,4,-5,6,7,8,9.3,10.4,11.5,12.6,13.7,true,null,null,CURRENT_TIME(),CURRENT_TIME(),CURRENT_DATE(),CURRENT_TIME(),'This is random textA','a','a','a') CREATE TABLE table_with_col_family (id BIGINT NOT NULL PRIMARY KEY, data.col1 VARCHAR) -UPSERT INTO table_with_col_family (id, col1) VALUES (1, 'test_row_1') \ No newline at end of file +UPSERT INTO table_with_col_family (id, col1) VALUES (1, 'test_row_1') +UPSERT INTO table_with_col_family (id, col1) VALUES (2, 'test_row_2') \ No newline at end of file diff --git a/phoenix5-spark3/src/it/resources/tenantSetup.sql b/phoenix5-spark3/src/it/resources/tenantSetup.sql index f62d8430..752dbbd4 100644 --- a/phoenix5-spark3/src/it/resources/tenantSetup.sql +++ b/phoenix5-spark3/src/it/resources/tenantSetup.sql @@ -16,3 +16,5 @@ CREATE VIEW IF NOT EXISTS TENANT_VIEW(TENANT_ONLY_COL VARCHAR) AS SELECT * FROM MULTITENANT_TEST_TABLE UPSERT INTO TENANT_VIEW (ORGANIZATION_ID, TENANT_ONLY_COL) VALUES ('defaultOrg', 'defaultData') +CREATE VIEW IF NOT EXISTS TENANT_VIEW_WITH_SALT(TENANT_ONLY_COL VARCHAR) AS SELECT * FROM MULTITENANT_TEST_TABLE_WITH_SALT +UPSERT INTO TENANT_VIEW_WITH_SALT (ORGANIZATION_ID, TENANT_ONLY_COL) VALUES ('defaultOrg', 'defaultData') diff --git a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala index 12e679b4..f217e46b 100644 --- a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala @@ -13,24 +13,25 @@ */ package org.apache.phoenix.spark -import java.sql.{Connection, DriverManager} -import java.util.Properties - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hbase.HConstants import org.apache.phoenix.query.BaseTest -import org.apache.phoenix.util.PhoenixRuntime -import org.apache.phoenix.util.ReadOnlyProps; -import org.apache.spark.sql.{SQLContext, SparkSession} -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.phoenix.spark.sql.connector.writer.PhoenixTestBatchWrite +import org.apache.phoenix.util.{PhoenixRuntime, ReadOnlyProps} +import org.apache.spark.sql.SparkSession import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite, Matchers} +import java.sql.{Connection, DriverManager} +import java.util.{Properties, TimeZone} + // Helper object to access the protected abstract static methods hidden in BaseTest object PhoenixSparkITHelper extends BaseTest { def getTestClusterConfig = new Configuration(BaseTest.config); def doSetup = { + // Set-up fixed timezone for DATE and TIMESTAMP tests + TimeZone.setDefault(TimeZone.getTimeZone("UTC")) // The @ClassRule doesn't seem to be getting picked up, force creation here before setup BaseTest.tmpFolder.create() BaseTest.setUpTestDriver(ReadOnlyProps.EMPTY_PROPS); @@ -49,8 +50,8 @@ object PhoenixSparkITHelper extends BaseTest { } /** - * Base class for PhoenixSparkIT - */ + * Base class for PhoenixSparkIT + */ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter with BeforeAndAfterAll { // A global tenantId we can use across tests @@ -64,15 +65,26 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter conf } - lazy val quorumAddress = { - ConfigurationUtil.getZookeeperURL(hbaseConfiguration).get + lazy val jdbcUrl = PhoenixSparkITHelper.getUrl + + lazy val quorumAddress = ConfigurationUtil.getZookeeperURL(hbaseConfiguration).get + + def getZookeeperURL(conf: Configuration): Option[String] = { + List( + Option(conf.get(HConstants.ZOOKEEPER_QUORUM)), + Option(conf.get(HConstants.ZOOKEEPER_CLIENT_PORT)), + Option(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)) + ).flatten match { + case Nil => None + case x: List[String] => Some(x.mkString(":")) + } } // Runs SQL commands located in the file defined in the sqlSource argument // Optional argument tenantId used for running tenant-specific SQL def setupTables(sqlSource: String, tenantId: Option[String]): Unit = { val props = new Properties - if(tenantId.isDefined) { + if (tenantId.isDefined) { props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId.get) } @@ -93,6 +105,11 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter conn.commit() } + before{ + // Reset batch counter after each test + PhoenixTestBatchWrite.TOTAL_BATCHES_COMMITTED_COUNT = 0 + } + override def beforeAll() { PhoenixSparkITHelper.doSetup diff --git a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala new file mode 100644 index 00000000..1251b858 --- /dev/null +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala @@ -0,0 +1,737 @@ +/* + Licensed 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.phoenix.spark + +import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil +import org.apache.phoenix.schema.types.{PSmallintArray, PVarchar} +import org.apache.phoenix.spark.sql.connector.reader.PhoenixTestPartitionReader +import org.apache.phoenix.spark.sql.connector.writer.PhoenixTestBatchWrite +import org.apache.phoenix.spark.sql.connector.{PhoenixDataSource, PhoenixTestingDataSource} +import org.apache.phoenix.util.{ColumnInfo, SchemaUtil} +import org.apache.spark.SparkException +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Row, SaveMode} + +import java.sql.DriverManager +import java.time.{LocalDate, ZoneId} +import scala.collection.mutable +import scala.collection.mutable.ListBuffer + +/** + * Note: If running directly from an IDE, these are the recommended VM parameters: + * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + */ +class PhoenixSparkDatasourceV1IT extends AbstractPhoenixSparkIT { + + test("Can persist data with case sensitive columns (like in avro schema)") { + val df = spark.createDataFrame( + Seq( + (1, 1, "test_child_1"), + (2, 1, "test_child_2"))). + // column names are case sensitive + toDF("ID", "TABLE3_ID", "t2col1") + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "TABLE3", + PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, + PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true")) + .mode(SaveMode.Overwrite) + .save() + + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3") + + val checkResults = List((1, 1, "test_child_1"), (2, 1, "test_child_2")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + + test("Can convert Phoenix schema") { + val phoenixSchema = List( + new ColumnInfo("varcharColumn", PVarchar.INSTANCE.getSqlType) + ) + + val catalystSchema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(phoenixSchema) + + val expected = new StructType(List(StructField("varcharColumn", StringType, nullable = true)).toArray) + + catalystSchema shouldEqual expected + } + + test("Can convert arrays of Short type in Phoenix schema") { + val phoenixSchema = List( + new ColumnInfo("arrayshortColumn", PSmallintArray.INSTANCE.getSqlType) + ) + + val catalystSchema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(phoenixSchema) + + val expected = new StructType(List(StructField("arrayshortColumn", ArrayType(ShortType, true), nullable = true)).toArray) + + catalystSchema shouldEqual expected + } + + test("Can create schema RDD and execute query") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("sql_table_1") + + val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df2.createOrReplaceTempView("sql_table_2") + + val sqlRdd = spark.sql( + """ + |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 + |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin + ) + + val count = sqlRdd.count() + + count shouldEqual 6L + } + + ignore("Ordering by pk columns should not require sorting") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + df1.createOrReplaceTempView("TABLE1") + + val sqlRdd = spark.sql("SELECT * FROM TABLE1 ORDER BY ID, COL1") + val plan = sqlRdd.queryExecution.sparkPlan + // verify the spark plan doesn't have a sort + assert(!plan.toString.contains("Sort")) + + val expectedResults = Array(Row.fromSeq(Seq(1, "test_row_1")), Row.fromSeq(Seq(2, "test_row_2"))) + val actual = sqlRdd.collect() + + actual shouldEqual expectedResults + } + + test("Verify correct number of partitions are created") { + val conn = DriverManager.getConnection(PhoenixSparkITHelper.getUrl) + val ddl = "CREATE TABLE SPLIT_TABLE (id VARCHAR NOT NULL PRIMARY KEY, val VARCHAR) split on ('e','j','o')" + conn.createStatement.execute(ddl) + val keys = Array("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s", + "t", "u", "v", "w", "x", "y", "z") + for (key <- keys) { + conn.createStatement.execute("UPSERT INTO SPLIT_TABLE VALUES('" + key + "', '" + key + "')") + } + conn.commit() + + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "SPLIT_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + df1.createOrReplaceTempView("SPLIT_TABLE") + val sqlRdd = spark.sql("SELECT * FROM SPLIT_TABLE") + val numPartitions = sqlRdd.rdd.partitions.size + + numPartitions shouldEqual 4 + } + + test("Can create schema RDD and execute query on case sensitive table (no config)") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("table4"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("table4") + + val sqlRdd = spark.sql("SELECT id FROM table4") + + val count = sqlRdd.count() + + count shouldEqual 2L + } + + test("Can create schema RDD and execute constrained query") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("sql_table_1") + + val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = 1") + + df2.createOrReplaceTempView("sql_table_2") + + val sqlRdd = spark.sql( + """ + |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 + |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin + ) + + val count = sqlRdd.count() + + count shouldEqual 1L + } + + test("Can create schema RDD with predicate that will never match") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = -1") + + df1.createOrReplaceTempView("table3") + + val sqlRdd = spark.sql("SELECT * FROM table3") + + val count = sqlRdd.count() + + count shouldEqual 0L + } + + test("Can create schema RDD with complex predicate") { + val predicate = "ID > 0 AND TIMESERIES_KEY BETWEEN " + + "CAST(TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP) AND " + + "CAST(TO_DATE('1990-01-30 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP)" + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "DATE_PREDICATE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + .filter(predicate) + + df1.createOrReplaceTempView("date_predicate_test_table") + + val sqlRdd = spark.sqlContext.sql("SELECT * FROM date_predicate_test_table") + + val count = sqlRdd.count() + + count shouldEqual 0L + } + + test("Can query an array table") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("ARRAY_TEST_TABLE") + + val sqlRdd = spark.sql("SELECT * FROM ARRAY_TEST_TABLE") + + val count = sqlRdd.count() + + // get row 0, column 1, which should be "VCARRAY" + val arrayValues = sqlRdd.collect().apply(0).apply(1) + + arrayValues should equal(Array("String1", "String2", "String3")) + + count shouldEqual 1L + } + + test("Can read a table as an RDD") { + val rdd1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + val count = rdd1.count() + + val arrayValues = rdd1.take(1)(0)(1) + + arrayValues should equal(Array("String1", "String2", "String3")) + + count shouldEqual 1L + } + + test("Can save to phoenix table from Spark without specifying all the columns") { + val dataSet = List(Row(1L, "1", 1), Row(2L, "2", 2), Row(3L, "3", 3)) + + // COL3 is missing both from the schema and from the dataset + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("COL1", StringType), + StructField("COL2", IntegerType))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT ID, COL1, COL2 FROM OUTPUT_TEST_TABLE") + val results = ListBuffer[Row]() + while (rs.next()) { + results.append(Row(rs.getLong(1), rs.getString(2), rs.getInt(3))) + } + + // Verify they match + (0 to results.size - 1).foreach { i => + dataSet(i) shouldEqual results(i) + } + } + + test("Can save dates to Phoenix using java.sql.Date") { + val date = java.sql.Date.valueOf("2016-09-30") + + // Since we are creating a Row we have to use java.sql.date + // java.util.date or joda.DateTime is not supported + val dataSet = Seq(Row(1L, "1", 1, date), Row(2L, "2", 2, date)) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("COL1", StringType), + StructField("COL2", IntegerType), + StructField("COL3", DateType))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT COL3 FROM OUTPUT_TEST_TABLE WHERE ID = 1 OR ID = 2 ORDER BY ID ASC") + val results = ListBuffer[java.sql.Date]() + while (rs.next()) { + results.append(rs.getDate(1)) + } + + // Verify the epochs are equal + results(0).getTime shouldEqual date.getTime + results(1).getTime shouldEqual date.getTime + } + + test("Can infer schema without defining columns") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + df.schema("ID").dataType shouldEqual LongType + df.schema("TABLE1_ID").dataType shouldEqual LongType + df.schema("t2col1").dataType shouldEqual StringType + } + + test("Spark SQL can use Phoenix as a data source with no schema specified") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + df.count() shouldEqual 2 + df.schema("ID").dataType shouldEqual LongType + df.schema("COL1").dataType shouldEqual StringType + } + + test("Can persist a dataframe") { + // Load from TABLE1 + val df = spark.sqlContext.read.format("org.apache.phoenix.spark").options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + // Save to TABLE1_COPY + df + .write + .format("org.apache.phoenix.spark") + .mode(SaveMode.Overwrite) + .option(PhoenixDataSource.TABLE, "TABLE1_COPY") + .option(PhoenixDataSource.ZOOKEEPER_URL, quorumAddress) + .save() + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE1_COPY") + + val checkResults = List((1L, "test_row_1"), (2, "test_row_2")) + val results = ListBuffer[(Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getString(2))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + + test("Can save arrays back to phoenix") { + val dataSet = List(Row(2L, Array("String1", "String2", "String3"))) + val schema = StructType(Seq( + StructField("ID", LongType, nullable = false), + StructField("VCARRAY", ArrayType(StringType, true)) + )) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT VCARRAY FROM ARRAY_TEST_TABLE WHERE ID = 2") + rs.next() + val sqlArray = rs.getArray(1).getArray().asInstanceOf[Array[String]] + + // Verify the arrays are equal + sqlArray shouldEqual dataSet(0).get(1) + } + + test("Can read from table with schema and escaped table name") { + // Manually escape + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + + var count = df1.count() + + count shouldEqual 1L + + // Use SchemaUtil + val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options( + Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load() + + count = df2.count() + + count shouldEqual 1L + } + + test("Ensure DataFrame field normalization (PHOENIX-2196)") { + val rdd1 = spark.sparkContext + .parallelize(Seq((1L, 1L, "One"), (2L, 2L, "Two"))) + .map(p => Row(p._1, p._2, p._3)) + + val schema = StructType(Seq( + StructField("id", LongType, nullable = false), + StructField("table1_id", LongType, nullable = true), + StructField("\"t2col1\"", StringType, nullable = true) + )) + + val df = spark.sqlContext.createDataFrame(rdd1, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + } + + test("Ensure Dataframe supports LIKE and IN filters (PHOENIX-2328)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark").options(Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + // Prefix match + val res1 = df.filter("COL1 like 'test_row_%'") + val plan = res1.groupBy().count().queryExecution.sparkPlan + res1.count() shouldEqual 2 + + // Suffix match + val res2 = df.filter("COL1 like '%_1'") + res2.count() shouldEqual 1 + res2.first.getString(1) shouldEqual "test_row_1" + + // Infix match + val res3 = df.filter("COL1 like '%_row_%'") + res3.count() shouldEqual 2 + + // Not like, match none + val res4 = df.filter("COL1 not like '%_row_%'") + res4.count() shouldEqual 0 + + // Not like, match all + val res5 = df.filter("COL1 not like '%_wor_%'") + res5.count() shouldEqual 2 + + // "IN", match all + val res6 = df.filter("COL1 in ('test_row_1', 'test_row_2')") + res6.count() shouldEqual 2 + + // "IN", match none + val res7 = df.filter("COL1 in ('foo', 'bar')") + res7.count() shouldEqual 0 + + // AND (and not again) + val res8 = df.filter("COL1 like '%_row_%' AND COL1 not like '%_1'") + res8.count() shouldEqual 1 + res8.first.getString(1) shouldEqual "test_row_2" + + // OR + val res9 = df.filter("COL1 like '%_1' OR COL1 like '%_2'") + res9.count() shouldEqual 2 + } + + test("Can load decimal types with accurate precision and scale (PHOENIX-2288)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "TEST_DECIMAL", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + assert(df.select("COL1").first().getDecimal(0) == BigDecimal("123.456789").bigDecimal) + } + + test("Can load small and tiny integer types (PHOENIX-2426)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "TEST_SMALL_TINY", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + assert(df.select("COL1").first().getShort(0).toInt == 32767) + assert(df.select("COL2").first().getByte(0).toInt == 127) + } + + test("Can save arrays from custom dataframes back to phoenix") { + val dataSet = List(Row(2L, Array("String1", "String2", "String3"), Array(1, 2, 3))) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("VCARRAY", ArrayType(StringType)), + StructField("INTARRAY", ArrayType(IntegerType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "ARRAYBUFFER_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT VCARRAY, INTARRAY FROM ARRAYBUFFER_TEST_TABLE WHERE ID = 2") + rs.next() + val stringArray = rs.getArray(1).getArray().asInstanceOf[Array[String]] + val intArray = rs.getArray(2).getArray().asInstanceOf[Array[Int]] + + // Verify the arrays are equal + stringArray shouldEqual dataSet(0).getAs[Array[String]](1) + intArray shouldEqual dataSet(0).getAs[Array[Int]](2) + } + + test("Can save arrays of AnyVal type back to phoenix") { + val dataSet = List(Row(2L, Array(1, 2, 3), Array(1L, 2L, 3L))) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("INTARRAY", ArrayType(IntegerType)), + StructField("BIGINTARRAY", ArrayType(LongType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "ARRAY_ANYVAL_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT INTARRAY, BIGINTARRAY FROM ARRAY_ANYVAL_TEST_TABLE WHERE ID = 2") + rs.next() + val intArray = rs.getArray(1).getArray().asInstanceOf[Array[Int]] + val longArray = rs.getArray(2).getArray().asInstanceOf[Array[Long]] + + // Verify the arrays are equal + intArray shouldEqual dataSet(0).get(1) + longArray shouldEqual dataSet(0).get(2) + } + + test("Can save arrays of Byte type back to phoenix") { + val dataSet = List(Row(2L, Array(1.toByte, 2.toByte, 3.toByte))) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("BYTEARRAY", ArrayType(ByteType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "ARRAY_BYTE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT BYTEARRAY FROM ARRAY_BYTE_TEST_TABLE WHERE ID = 2") + rs.next() + val byteArray = rs.getArray(1).getArray().asInstanceOf[Array[Byte]] + + // Verify the arrays are equal + byteArray shouldEqual dataSet(0).get(1) + } + + test("Can save arrays of Short type back to phoenix") { + val dataSet = List(Row(2L, Array(1.toShort, 2.toShort, 3.toShort))) + + val schema = StructType( + Seq(StructField("ID", LongType, nullable = false), + StructField("SHORTARRAY", ArrayType(ShortType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "ARRAY_SHORT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT SHORTARRAY FROM ARRAY_SHORT_TEST_TABLE WHERE ID = 2") + rs.next() + val shortArray = rs.getArray(1).getArray().asInstanceOf[Array[Short]] + + // Verify the arrays are equal + shortArray shouldEqual dataSet(0).get(1) + } + + test("Can save binary types back to phoenix") { + val dataSet = List(Row(2L, Array[Byte](1), Array[Byte](1, 2, 3), Array[Array[Byte]](Array[Byte](1), Array[Byte](2)))) + + val schema = StructType( + Seq(StructField("ID", LongType, false), + StructField("BIN", BinaryType), + StructField("VARBIN", BinaryType), + StructField("BINARRAY", ArrayType(BinaryType)))) + + val rowRDD = spark.sparkContext.parallelize(dataSet) + + // Apply the schema to the RDD. + val df = spark.sqlContext.createDataFrame(rowRDD, schema) + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "VARBINARY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + // Load the results back + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT BIN, VARBIN, BINARRAY FROM VARBINARY_TEST_TABLE WHERE ID = 2") + rs.next() + val byte = rs.getBytes("BIN") + val varByte = rs.getBytes("VARBIN") + val varByteArray = rs.getArray("BINARRAY").getArray().asInstanceOf[Array[Array[Byte]]] + + // Verify the arrays are equal + byte shouldEqual dataSet(0).get(1) + varByte shouldEqual dataSet(0).get(2) + varByteArray shouldEqual dataSet(0).get(3) + } + + test("Can load and filter Phoenix DATE columns through DataFrame API") { + val df = spark.sqlContext.read + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + val dt = df.select("COL1").first().getDate(0).getTime + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli + + assert(expected == dt) + + df.createOrReplaceTempView("DATE_TEST") + val df2 = spark.sql("SELECT * FROM DATE_TEST WHERE COL1 > TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss')") + assert(df2.count() == 1L) + } + + test("Filter operation doesn't work for column names containing a white space (PHOENIX-2547)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("space"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + val res = df.filter(df.col("first name").equalTo("xyz")) + // Make sure we got the right value back + assert(res.collectAsList().size() == 1L) + } + + test("Spark Phoenix cannot recognize Phoenix view fields (PHOENIX-2290)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + df.createOrReplaceTempView("temp") + + // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions + // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle + + val res1 = spark.sql("select * from temp where salary = '10000' ") + assert(res1.collectAsList().size() == 1L) + + val res2 = spark.sql("select * from temp where \"salary\" = '10000' ") + assert(res2.collectAsList().size() == 0L) + + val res3 = spark.sql("select * from temp where salary > '10000' ") + assert(res3.collectAsList().size() == 2L) + } + + test("Queries with small case column-names return empty result-set when working with Spark Datasource Plugin (PHOENIX-2336)") { + val df = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + + // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions + // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle + + val res1 = df.filter(df.col("first name").equalTo("foo")) + assert(res1.collectAsList().size() == 1L) + + val res2 = df.filter("\"first name\" = 'foo'") + assert(res2.collectAsList().size() == 0L) + + val res3 = df.filter("salary = '10000'") + assert(res3.collectAsList().size() == 1L) + + val res4 = df.filter("salary > '10000'") + assert(res4.collectAsList().size() == 2L) + } + + test("Can coerce Phoenix DATE columns to TIMESTAMP through DataFrame API") { + val df = spark.sqlContext.read + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, PhoenixDataSource.DATE_AS_TIME_STAMP -> "true")) + .load + val dtRes = df.select("COL1").first() + val ts = dtRes.getTimestamp(0).getTime + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli + + assert(expected == ts) + } + + test("Can load Phoenix Time columns through DataFrame API") { + val df = spark.sqlContext.read + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "TIME_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + val time = df.select("COL1").first().getTimestamp(0).getTime + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli + + assert(expected == time) + } + + test("can read all Phoenix data types") { + val df = spark.sqlContext.read + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .load + + df.write + .format("org.apache.phoenix.spark") + .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .mode(SaveMode.Overwrite) + .save() + + df.count() shouldEqual 1 + } + +} diff --git a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala new file mode 100644 index 00000000..28907dbc --- /dev/null +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala @@ -0,0 +1,117 @@ +/* + Licensed 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.phoenix.spark + +import scala.collection.mutable.ListBuffer + +/** + * Sub-class of PhoenixSparkIT used for tenant-specific tests + * + * Note: All schema related variables (table name, column names, default data, etc) are coupled with + * phoenix-spark/src/it/resources/tenantSetup.sql + * + * Note: If running directly from an IDE, these are the recommended VM parameters: + * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + * + */ +class PhoenixSparkDatasourceV1ITTenantSpecific extends AbstractPhoenixSparkIT { + + // Tenant-specific schema info + val OrgIdCol = "ORGANIZATION_ID" + val TenantOnlyCol = "TENANT_ONLY_COL" + val TenantTable = "TENANT_VIEW" + + // Data set for tests that write to Phoenix + val TestDataSet = List(("testOrg1", "data1"), ("testOrg2", "data2"), ("testOrg3", "data3")) + val TestDataSet2 = List(("testOrg1", "data1", TenantId, "g1"), ("testOrg2", "data2", TenantId, "g3"), + ("testOrg3", "data3", TenantId, "g3")) + + /** + * Helper method used by write tests to verify content written. + * Assumes the caller has written the TestDataSet (defined above) to Phoenix + * and that 1 row of default data exists (upserted after table creation in tenantSetup.sql) + */ + def verifyResults(): Unit = { + // Contains the default data upserted into the tenant-specific table in tenantSetup.sql and the data upserted by tests + val VerificationDataSet = List(("defaultOrg", "defaultData")) ::: TestDataSet + + val SelectStatement = "SELECT " + OrgIdCol + "," + TenantOnlyCol + " FROM " + TenantTable + val stmt = conn.createStatement() + val rs = stmt.executeQuery(SelectStatement) + + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual VerificationDataSet + } + + /*****************/ + /** Read tests **/ + /*****************/ + + test("Can read from tenant-specific table as DataFrame") { + val df = spark.sqlContext.phoenixTableAsDataFrame( + TenantTable, + Seq(OrgIdCol, TenantOnlyCol), + zkUrl = Some(quorumAddress), + tenantId = Some(TenantId), + conf = hbaseConfiguration) + + // There should only be 1 row upserted in tenantSetup.sql + val count = df.count() + count shouldEqual 1L + } + + test("Can read from tenant-specific table as RDD") { + val rdd = spark.sparkContext.phoenixTableAsRDD( + TenantTable, + Seq(OrgIdCol, TenantOnlyCol), + zkUrl = Some(quorumAddress), + tenantId = Some(TenantId), + conf = hbaseConfiguration) + + // There should only be 1 row upserted in tenantSetup.sql + val count = rdd.count() + count shouldEqual 1L + } + + /*****************/ + /** Write tests **/ + /*****************/ + + test("Can write a DataFrame using 'DataFrame.saveToPhoenix' to tenant-specific view") { + val sqlContext = spark.sqlContext + import sqlContext.implicits._ + + val df = spark.sparkContext.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol) + df.saveToPhoenix(TenantTable, zkUrl = Some(quorumAddress), tenantId = Some(TenantId)) + + verifyResults + } + + test("Can write an RDD to Phoenix tenant-specific view") { + spark.sparkContext + .parallelize(TestDataSet) + .saveToPhoenix( + TenantTable, + Seq(OrgIdCol, TenantOnlyCol), + hbaseConfiguration, + tenantId = Some(TenantId) + ) + + verifyResults + } +} diff --git a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala index 7a16b005..f0ff9557 100644 --- a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala @@ -18,7 +18,7 @@ import java.sql.DriverManager import java.util.Date import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil import org.apache.phoenix.query.QueryServices -import org.apache.phoenix.schema.types.{PSmallintArray, PUnsignedSmallintArray, PVarchar} +import org.apache.phoenix.schema.types.{PSmallintArray, PVarchar} import org.apache.phoenix.spark.sql.connector.{PhoenixDataSource, PhoenixTestingDataSource} import org.apache.phoenix.spark.sql.connector.reader.PhoenixTestPartitionReader import org.apache.phoenix.spark.sql.connector.writer.PhoenixTestBatchWrite @@ -27,6 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.types.{ArrayType, BinaryType, ByteType, DateType, IntegerType, LongType, ShortType, StringType, StructField, StructType} import org.apache.spark.sql.{Row, SaveMode} +import java.time.{LocalDate, ZoneId} import scala.collection.mutable import scala.collection.mutable.ListBuffer @@ -45,8 +46,9 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { toDF("ID", "TABLE3_ID", "t2col1") df.write .format("phoenix") - .options(Map("table" -> "TABLE3", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true")) + .options(Map(PhoenixDataSource.TABLE -> "TABLE3", + PhoenixDataSource.JDBC_URL -> jdbcUrl, + PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true")) .mode(SaveMode.Append) .save() @@ -65,29 +67,6 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { results.toList shouldEqual checkResults } - // INSERT is not support using DataSource v2 api yet - ignore("Can use write data using spark SQL INSERT") { - val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE3", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - df1.createOrReplaceTempView("TABLE3") - - // Insert data - spark.sql("INSERT INTO TABLE3 VALUES(10, 10, 10)") - spark.sql("INSERT INTO TABLE3 VALUES(20, 20, 20)") - - // Verify results - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT * FROM TABLE3 WHERE ID>=10") - val expectedResults = List((10, 10, "10"), (20, 20, "20")) - val results = ListBuffer[(Long, Long, String)]() - while (rs.next()) { - results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) - } - stmt.close() - - results.toList shouldEqual expectedResults - } - test("Can persist data into transactional tables with phoenix.transactions.enabled option") { var extraOptions = QueryServices.TRANSACTIONS_ENABLED + "=true"; val df = spark.createDataFrame( @@ -98,8 +77,8 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { toDF("ID", "TABLE5_ID", "t5col1") df.write .format("phoenix") - .options(Map("table" -> "TABLE5", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true", + .options(Map(PhoenixDataSource.TABLE -> "TABLE5", + PhoenixDataSource.JDBC_URL -> jdbcUrl, PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true", PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)) .mode(SaveMode.Append) .save() @@ -134,8 +113,8 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { toDF("ID", "TABLE5_ID", "t5col1") df.write .format("phoenix") - .options(Map("table" -> "TABLE5", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true", + .options(Map(PhoenixDataSource.TABLE -> "TABLE5", + PhoenixDataSource.JDBC_URL -> jdbcUrl, PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true", PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)) .mode(SaveMode.Append) .save() @@ -185,12 +164,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can create schema RDD and execute query") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("sql_table_1") val df2 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df2.createOrReplaceTempView("sql_table_2") @@ -207,7 +186,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { ignore("Ordering by pk columns should not require sorting") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("TABLE1") val sqlRdd = spark.sql("SELECT * FROM TABLE1 ORDER BY ID, COL1") @@ -233,7 +212,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { conn.commit() val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "SPLIT_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> "SPLIT_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("SPLIT_TABLE") val sqlRdd = spark.sql("SELECT * FROM SPLIT_TABLE") val numPartitions = sqlRdd.rdd.partitions.size @@ -243,7 +222,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can create schema RDD and execute query on case sensitive table (no config)") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> SchemaUtil.getEscapedArgument("table4"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("table4"), PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("table4") @@ -256,12 +235,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can create schema RDD and execute constrained query") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("sql_table_1") val df2 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = 1") + .options( Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load.filter("ID = 1") df2.createOrReplaceTempView("sql_table_2") @@ -278,7 +257,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can create schema RDD with predicate that will never match") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = -1") + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load.filter("ID = -1") df1.createOrReplaceTempView("table3") @@ -294,7 +273,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { "CAST(TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP) AND " + "CAST(TO_DATE('1990-01-30 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP)" val df1 = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> "DATE_PREDICATE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "DATE_PREDICATE_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load .filter(predicate) @@ -309,7 +288,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can query an array table") { val df1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> "ARRAY_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df1.createOrReplaceTempView("ARRAY_TEST_TABLE") @@ -327,7 +306,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can read a table as an RDD") { val rdd1 = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> "ARRAY_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load val count = rdd1.count() @@ -343,7 +322,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { var extraOptions = PhoenixTestPartitionReader.RETURN_NULL_CURR_ROW + "=true" var rdd = spark.sqlContext.read .format(PhoenixTestingDataSource.TEST_SOURCE) - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl, PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)).load // Expect to get a NullPointerException in the executors @@ -356,7 +335,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { extraOptions = PhoenixTestPartitionReader.RETURN_NULL_CURR_ROW + "=false" rdd = spark.sqlContext.read .format(PhoenixTestingDataSource.TEST_SOURCE) - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl, PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)).load val stringValue = rdd.take(2)(0)(1) stringValue shouldEqual "test_row_1" @@ -378,7 +357,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() @@ -428,7 +407,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { PhoenixTestBatchWrite.TOTAL_BATCHES_COMMITTED_COUNT shouldEqual 0 df.write .format(PhoenixTestingDataSource.TEST_SOURCE) - .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, + .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl, PhoenixDataSource.PHOENIX_CONFIGS -> extraOptions)) .mode(SaveMode.Append) .save() @@ -457,7 +436,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() @@ -476,7 +455,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can infer schema without defining columns") { val df = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + .options( Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() df.schema("ID").dataType shouldEqual LongType df.schema("TABLE1_ID").dataType shouldEqual LongType df.schema("t2col1").dataType shouldEqual StringType @@ -484,7 +463,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Spark SQL can use Phoenix as a data source with no schema specified") { val df = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load df.count() shouldEqual 2 df.schema("ID").dataType shouldEqual LongType df.schema("COL1").dataType shouldEqual StringType @@ -494,7 +473,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { // The easily parsed execution plan is only logged to stdout, but is not accessible from the objects. ignore("Datasource v2 pushes down filters") { val df = spark.sqlContext.read.format("phoenix") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load val res = df.filter(df("COL1") === "test_row_1" && df("ID") === 1L).select(df("ID")) // Make sure we got the right value back @@ -509,15 +488,15 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can persist a dataframe") { // Load from TABLE1 - val df = spark.sqlContext.read.format("phoenix").options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + val df = spark.sqlContext.read.format("phoenix").options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load // Save to TABLE1_COPY df .write .format("phoenix") .mode(SaveMode.Append) - .option("table", "TABLE1_COPY") - .option(PhoenixDataSource.ZOOKEEPER_URL, quorumAddress) + .option(PhoenixDataSource.TABLE, "TABLE1_COPY") + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) .save() // Verify results @@ -548,7 +527,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "ARRAY_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() @@ -565,7 +544,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can read from table with schema and escaped table name") { // Manually escape val df1 = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + .options(Map(PhoenixDataSource.TABLE -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() var count = df1.count() @@ -574,7 +553,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { // Use SchemaUtil val df2 = spark.sqlContext.read.format("phoenix") .options( - Map("table" -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load() count = df2.count() @@ -597,13 +576,13 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() } test("Ensure Dataframe supports LIKE and IN filters (PHOENIX-2328)") { - val df = spark.sqlContext.read.format("phoenix").options(Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + val df = spark.sqlContext.read.format("phoenix").options(Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() // Prefix match val res1 = df.filter("COL1 like 'test_row_%'") val plan = res1.groupBy().count().queryExecution.sparkPlan @@ -646,13 +625,13 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can load decimal types with accurate precision and scale (PHOENIX-2288)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> "TEST_DECIMAL", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + .options(Map(PhoenixDataSource.TABLE -> "TEST_DECIMAL", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() assert(df.select("COL1").first().getDecimal(0) == BigDecimal("123.456789").bigDecimal) } test("Can load small and tiny integer types (PHOENIX-2426)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> "TEST_SMALL_TINY", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() + .options(Map(PhoenixDataSource.TABLE -> "TEST_SMALL_TINY", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() assert(df.select("COL1").first().getShort(0).toInt == 32767) assert(df.select("COL2").first().getByte(0).toInt == 127) } @@ -672,7 +651,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAYBUFFER_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "ARRAYBUFFER_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() @@ -703,7 +682,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAY_ANYVAL_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "ARRAY_ANYVAL_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() @@ -733,7 +712,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAY_BYTE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "ARRAY_BYTE_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() @@ -761,7 +740,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "ARRAY_SHORT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "ARRAY_SHORT_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() @@ -791,7 +770,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { df.write .format("phoenix") - .options(Map("table" -> "VARBINARY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "VARBINARY_TEST_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() @@ -812,17 +791,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can load and filter Phoenix DATE columns through DataFrame API") { val df = spark.sqlContext.read .format("phoenix") - .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "DATE_TEST", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load val dt = df.select("COL1").first().getDate(0).getTime - val epoch = new Date().getTime - - // NOTE: Spark DateType drops hour, minute, second, as per the java.sql.Date spec - // Use 'dateAsTimestamp' option to coerce DATE to TIMESTAMP without losing resolution + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - // Note that Spark also applies the timezone offset to the returned date epoch. Rather than perform timezone - // gymnastics, just make sure we're within 24H of the epoch generated just now - assert(Math.abs(epoch - dt) < 86400000) + assert(expected == dt) df.createOrReplaceTempView("DATE_TEST") val df2 = spark.sql("SELECT * FROM DATE_TEST WHERE COL1 > TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss')") @@ -831,7 +805,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Filter operation doesn't work for column names containing a white space (PHOENIX-2547)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> SchemaUtil.getEscapedArgument("space"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("space"), PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load val res = df.filter(df.col("first name").equalTo("xyz")) // Make sure we got the right value back @@ -840,7 +814,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Spark Phoenix cannot recognize Phoenix view fields (PHOENIX-2290)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load df.createOrReplaceTempView("temp") @@ -859,7 +833,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Queries with small case column-names return empty result-set when working with Spark Datasource Plugin (PHOENIX-2336)") { val df = spark.sqlContext.read.format("phoenix") - .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions @@ -881,34 +855,35 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can coerce Phoenix DATE columns to TIMESTAMP through DataFrame API") { val df = spark.sqlContext.read .format("phoenix") - .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, "dateAsTimestamp" -> "true")) + .options(Map(PhoenixDataSource.TABLE -> "DATE_TEST", PhoenixDataSource.JDBC_URL -> jdbcUrl, PhoenixDataSource.DATE_AS_TIME_STAMP -> "true")) .load val dtRes = df.select("COL1").first() val ts = dtRes.getTimestamp(0).getTime - val epoch = new Date().getTime + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - assert(Math.abs(epoch - ts) < 300000) + assert(expected == ts) } test("Can load Phoenix Time columns through DataFrame API") { val df = spark.sqlContext.read .format("phoenix") - .options(Map("table" -> "TIME_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "TIME_TEST", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load val time = df.select("COL1").first().getTimestamp(0).getTime - val epoch = new Date().getTime - assert(Math.abs(epoch - time) < 86400000) + val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli + + assert(expected == time) } test("can read all Phoenix data types") { val df = spark.sqlContext.read .format("phoenix") - .options(Map("table" -> "GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "GIGANTIC_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load df.write .format("phoenix") - .options(Map("table" -> "OUTPUT_GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) + .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_GIGANTIC_TABLE", PhoenixDataSource.JDBC_URL -> jdbcUrl)) .mode(SaveMode.Append) .save() @@ -938,9 +913,9 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can read data and map column to columnName") { val df = spark.read.format("phoenix") .options( - Map("table" -> SchemaUtil.getEscapedArgument("TABLE_WITH_COL_FAMILY"), + Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("TABLE_WITH_COL_FAMILY"), "doNotMapColumnFamily" -> "true", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + PhoenixDataSource.JDBC_URL -> jdbcUrl)).load val schema = df.schema @@ -955,9 +930,9 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { test("Can read data and map column to colFamily.columnName") { val df = spark.read.format("phoenix") .options( - Map("table" -> SchemaUtil.getEscapedArgument("TABLE_WITH_COL_FAMILY"), + Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("TABLE_WITH_COL_FAMILY"), "doNotMapColumnFamily" -> "false", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + PhoenixDataSource.JDBC_URL -> jdbcUrl)).load val schema = df.schema diff --git a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala index 7a9b2180..d4271dc2 100644 --- a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala @@ -13,141 +13,161 @@ */ package org.apache.phoenix.spark -import org.apache.phoenix.util.PhoenixRuntime -import org.apache.spark.sql.SQLContext +import org.apache.phoenix.spark.sql.connector.PhoenixDataSource +import org.apache.spark.sql.{Row, SaveMode} import scala.collection.mutable.ListBuffer /** - * Sub-class of PhoenixSparkIT used for tenant-specific tests - * - * Note: All schema related variables (table name, column names, default data, etc) are coupled with - * phoenix-spark/src/it/resources/tenantSetup.sql - * - * Note: If running directly from an IDE, these are the recommended VM parameters: - * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m - * - */ + * Sub-class of PhoenixSparkIT used for tenant-specific tests + * + * Note: All schema related variables (table name, column names, default data, etc) are coupled with + * phoenix-spark/src/it/resources/tenantSetup.sql + * + * Note: If running directly from an IDE, these are the recommended VM parameters: + * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + * + */ class PhoenixSparkITTenantSpecific extends AbstractPhoenixSparkIT { // Tenant-specific schema info val OrgIdCol = "ORGANIZATION_ID" val TenantOnlyCol = "TENANT_ONLY_COL" val TenantTable = "TENANT_VIEW" + val TenantTableWithSalt = "TENANT_VIEW_WITH_SALT" // Data set for tests that write to Phoenix val TestDataSet = List(("testOrg1", "data1"), ("testOrg2", "data2"), ("testOrg3", "data3")) - val TestDataSet2 = List(("testOrg1", "data1", TenantId, "g1"), ("testOrg2", "data2", TenantId, "g3"), - ("testOrg3", "data3", TenantId, "g3")) - - /** - * Helper method used by write tests to verify content written. - * Assumes the caller has written the TestDataSet (defined above) to Phoenix - * and that 1 row of default data exists (upserted after table creation in tenantSetup.sql) - */ - def verifyResults(): Unit = { - // Contains the default data upserted into the tenant-specific table in tenantSetup.sql and the data upserted by tests - val VerificationDataSet = List(("defaultOrg", "defaultData")) ::: TestDataSet - - val SelectStatement = "SELECT " + OrgIdCol + "," + TenantOnlyCol + " FROM " + TenantTable - val stmt = conn.createStatement() - val rs = stmt.executeQuery(SelectStatement) + val TestDataSet2 = List(("testOrg4", "data4"), ("testOrg5", "data5")) - val results = ListBuffer[(String, String)]() - while (rs.next()) { - results.append((rs.getString(1), rs.getString(2))) - } - stmt.close() - results.toList shouldEqual VerificationDataSet + val sqlTableName = "TENANT_TABLE" + + after { + spark.sql(s"DROP TABLE IF EXISTS $sqlTableName") } - /*****************/ - /** Read tests **/ - /*****************/ + /** ************** */ + /** Read tests * */ + /** ************** */ test("Can read from tenant-specific table as DataFrame") { - val df = spark.sqlContext.phoenixTableAsDataFrame( - TenantTable, - Seq(OrgIdCol, TenantOnlyCol), - zkUrl = Some(quorumAddress), - tenantId = Some(TenantId), - conf = hbaseConfiguration) + val expected = Array(Row.fromSeq(Seq("defaultOrg", "defaultData"))) + val df = spark.read + .format("phoenix") + .option(PhoenixDataSource.TABLE, TenantTable) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .option(PhoenixDataSource.TENANT_ID, TenantId) + .load() + .select(OrgIdCol, TenantOnlyCol) // There should only be 1 row upserted in tenantSetup.sql - val count = df.count() - count shouldEqual 1L + val result = df.collect() + expected shouldEqual result } - test("Can read from tenant-specific table as RDD") { - val rdd = spark.sparkContext.phoenixTableAsRDD( - TenantTable, - Seq(OrgIdCol, TenantOnlyCol), - zkUrl = Some(quorumAddress), - tenantId = Some(TenantId), - conf = hbaseConfiguration) + test("Can read from tenant-specific and salted table as DataFrame") { + val expected = Array(Row.fromSeq(Seq("defaultOrg", "defaultData"))) + val df = spark.read + .format("phoenix") + .option(PhoenixDataSource.TABLE, TenantTableWithSalt) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .option(PhoenixDataSource.TENANT_ID, TenantId) + .load() + .select(OrgIdCol, TenantOnlyCol) // There should only be 1 row upserted in tenantSetup.sql - val count = rdd.count() - count shouldEqual 1L + val result = df.collect() + expected shouldEqual result } - /*****************/ - /** Write tests **/ - /*****************/ + test("Can read from tenant table using spark-sql") { - test("Can write a DataFrame using 'DataFrame.saveToPhoenix' to tenant-specific view") { - val sqlContext = spark.sqlContext - import sqlContext.implicits._ + val expected = Array(Row.fromSeq(Seq("defaultOrg", "defaultData"))) - val df = spark.sparkContext.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol) - df.saveToPhoenix(TenantTable, zkUrl = Some(quorumAddress), tenantId = Some(TenantId)) + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' '$TenantTable', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', '${PhoenixDataSource.TENANT_ID}' '${TenantId}')") + + val dataFrame = spark.sql(s"SELECT $OrgIdCol,$TenantOnlyCol FROM $sqlTableName") - verifyResults + dataFrame.collect() shouldEqual expected } - ignore("Can write a DataFrame using 'DataFrame.write' to tenant-specific view - Spark2 sparse columns") { + /** ************** */ + /** Write tests * */ + /** ************** */ + + test("Can write a DataFrame using 'DataFrame.write' to tenant-specific view") { val sqlContext = spark.sqlContext import sqlContext.implicits._ - val df = spark.sparkContext.parallelize(TestDataSet2).toDF(OrgIdCol, TenantOnlyCol); - + val df = spark.sparkContext.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol) df.write .format("phoenix") - .mode("overwrite") - .option("table", TenantTable) - .option(PhoenixRuntime.TENANT_ID_ATTRIB, TenantId) - .option("zkUrl", PhoenixSparkITHelper.getUrl) + .mode(SaveMode.Append) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .option(PhoenixDataSource.TABLE, TenantTable) + .option(PhoenixDataSource.TENANT_ID, TenantId) .save() - verifyResults + val expected = List(("defaultOrg", "defaultData")) ::: TestDataSet + val SelectStatement = s"SELECT $OrgIdCol,$TenantOnlyCol FROM $TenantTable" + val stmt = conn.createStatement() + val rs = stmt.executeQuery(SelectStatement) + + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual expected } - test("Can write a DataFrame using 'DataFrame.write' to tenant-specific view - Spark3 all columns") { + test("Can write a DataFrame using 'DataFrame.write' to tenant-specific with salt view") { val sqlContext = spark.sqlContext import sqlContext.implicits._ - val df = spark.sparkContext.parallelize(TestDataSet2).toDF(OrgIdCol, TenantOnlyCol, "TENANT_ID", "GLOBAL_COL1") - + val df = spark.sparkContext.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol) df.write .format("phoenix") - .mode("append") - .option("table", TenantTable) - .option(PhoenixRuntime.TENANT_ID_ATTRIB, TenantId) - .option("zkUrl", PhoenixSparkITHelper.getUrl) + .mode(SaveMode.Append) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .option(PhoenixDataSource.TABLE, TenantTableWithSalt) + .option(PhoenixDataSource.TENANT_ID, TenantId) .save() + val expected = List(("defaultOrg", "defaultData")) ::: TestDataSet + val SelectStatement = s"SELECT $OrgIdCol,$TenantOnlyCol FROM $TenantTableWithSalt" + val stmt = conn.createStatement() + val rs = stmt.executeQuery(SelectStatement) + + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual expected } - test("Can write an RDD to Phoenix tenant-specific view") { - spark.sparkContext - .parallelize(TestDataSet) - .saveToPhoenix( - TenantTable, - Seq(OrgIdCol, TenantOnlyCol), - hbaseConfiguration, - tenantId = Some(TenantId) - ) - - verifyResults + test("Can use write data into tenant table using spark SQL INSERT") { + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' '$TenantTable', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', '${PhoenixDataSource.TENANT_ID}' '${TenantId}')") + + (TestDataSet ::: TestDataSet2).foreach(tuple => { + // Insert data + spark.sql(s"INSERT INTO $sqlTableName VALUES('${tuple._1}', NULL, '${tuple._2}')") + }) + + val expected = List(("defaultOrg", "defaultData")) ::: TestDataSet ::: TestDataSet2 + val SelectStatement = s"SELECT $OrgIdCol,$TenantOnlyCol FROM $TenantTable" + val stmt = conn.createStatement() + val rs = stmt.executeQuery(SelectStatement) + + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual expected } + } diff --git a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala new file mode 100644 index 00000000..c14252c3 --- /dev/null +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala @@ -0,0 +1,113 @@ +package org.apache.phoenix.spark + +import org.apache.phoenix.spark.sql.connector.PhoenixDataSource +import org.apache.spark.sql.Row + +import scala.collection.mutable.ListBuffer + +/** + * Note: If running directly from an IDE, these are the recommended VM parameters: + * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + */ +class PhoenixSparkSqlIT extends AbstractPhoenixSparkIT { + val sqlTableName = "SQL_TABLE" + + after { + spark.sql(s"DROP TABLE IF EXISTS $sqlTableName") + } + + test("Can read from table using spark-sql") { + val expected : Array[Row] = Array( + Row.fromSeq(Seq(1, "test_row_1")), + Row.fromSeq(Seq(2, "test_row_2")) + ) + + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE1', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl')") + + val dataFrame = spark.sql(s"SELECT * FROM $sqlTableName") + + dataFrame.collect() shouldEqual expected + } + + test("Can read from table using spark-sql with where clause and selecting specific columns`") { + val expected : Array[Row] = Array( + Row.fromSeq(Seq("test_row_1")) + ) + + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE1', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl')") + + val dataFrame = spark.sql(s"SELECT COL1 FROM $sqlTableName where COL1='test_row_1'") + + dataFrame.collect() shouldEqual expected + } + + test("Can read from table having column family name") { + val expected : Array[Row] = Array( + Row.fromSeq(Seq(1)) + ) + + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE_WITH_COL_FAMILY', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', 'doNotMapColumnFamily' 'false')") + val dataFrame = spark.sql(s"SELECT ID FROM $sqlTableName where `DATA.COL1`='test_row_1'") + + dataFrame.collect() shouldEqual expected + } + + test("Can read from table having column family name and map column to `columnName`") { + val expected : Array[Row] = Array( + Row.fromSeq(Seq(1)) + ) + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE_WITH_COL_FAMILY', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', 'doNotMapColumnFamily' 'true')") + val dataFrame = spark.sql(s"SELECT ID FROM $sqlTableName where COL1='test_row_1'") + + dataFrame.collect() shouldEqual expected + } + + test("Can use write data using spark SQL INSERT") { + spark.sql(s"CREATE TABLE $sqlTableName USING phoenix " + + s"OPTIONS ('table' 'TABLE3', '${PhoenixDataSource.JDBC_URL}' '$jdbcUrl', 'skipNormalizingIdentifier' 'true')") + + // Insert data + spark.sql(s"INSERT INTO $sqlTableName VALUES(10, 10, '10')") + spark.sql(s"INSERT INTO $sqlTableName VALUES(20, 20, '20')") + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3 WHERE ID>=10") + val expectedResults = List((10, 10, "10"), (20, 20, "20")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual expectedResults + } + + // INSERT using dataFrame as init to spark-sql + ignore("Can use write data using spark SQL INSERT using dataframe createOrReplaceTempView") { + val df1 = spark.sqlContext.read.format("phoenix") + .options( Map("table" -> "TABLE3", PhoenixDataSource.JDBC_URL -> this.jdbcUrl)).load + df1.createOrReplaceTempView("TABLE3") + + // Insert data + spark.sql("INSERT INTO TABLE3 VALUES(10, 10, 10)") + spark.sql("INSERT INTO TABLE3 VALUES(20, 20, 20)") + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3 WHERE ID>=10") + val expectedResults = List((10, 10, "10"), (20, 20, "20")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual expectedResults + } + +} diff --git a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/PhoenixDataSource.java b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/PhoenixDataSource.java index d2a7410e..37f942ae 100644 --- a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/PhoenixDataSource.java +++ b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/PhoenixDataSource.java @@ -17,9 +17,15 @@ */ package org.apache.phoenix.spark.sql.connector; +import org.apache.phoenix.schema.PColumn; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.spark.SparkSchemaUtil; +import org.apache.phoenix.util.SchemaUtil; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.sources.BaseRelation; +import org.apache.spark.sql.sources.RelationProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.phoenix.spark.SparkSchemaUtil; import org.apache.phoenix.util.ColumnInfo; import org.apache.phoenix.util.PhoenixRuntime; import org.apache.spark.sql.connector.catalog.Table; @@ -34,9 +40,7 @@ import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; -import java.util.List; -import java.util.Map; -import java.util.Properties; +import java.util.*; import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL; import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR; @@ -44,7 +48,7 @@ /** * Implements the DataSourceV2 api to read and write from Phoenix tables */ -public class PhoenixDataSource implements TableProvider, DataSourceRegister { +public class PhoenixDataSource implements TableProvider, DataSourceRegister, RelationProvider { private static final Logger logger = LoggerFactory.getLogger(PhoenixDataSource.class); public static final String SKIP_NORMALIZING_IDENTIFIER = "skipNormalizingIdentifier"; @@ -52,19 +56,22 @@ public class PhoenixDataSource implements TableProvider, DataSourceRegister { public static final String ZOOKEEPER_URL = "zkUrl"; public static final String JDBC_URL = "jdbcUrl"; public static final String PHOENIX_CONFIGS = "phoenixconfigs"; - protected StructType schema; + public static final String TABLE = "table"; + public static final String DATE_AS_TIME_STAMP = "dateAsTimestamp"; + public static final String DO_NOT_MAP_COLUMN_FAMILY = "doNotMapColumnFamily"; + public static final String TENANT_ID = "TenantId"; @Override - public StructType inferSchema(CaseInsensitiveStringMap options){ - if (options.get("table") == null) { + public StructType inferSchema(CaseInsensitiveStringMap options) { + if (options.get(TABLE) == null) { throw new RuntimeException("No Phoenix option " + "Table" + " defined"); } String jdbcUrl = getJdbcUrlFromOptions(options); - String tableName = options.get("table"); - String tenant = options.get(PhoenixRuntime.TENANT_ID_ATTRIB); - boolean dateAsTimestamp = Boolean.parseBoolean(options.getOrDefault("dateAsTimestamp", Boolean.toString(false))); - boolean doNotMapColumnFamily = Boolean.parseBoolean(options.getOrDefault("doNotMapColumnFamily", Boolean.toString(false))); + String tableName = options.get(TABLE); + String tenant = options.get(TENANT_ID); + boolean dateAsTimestamp = Boolean.parseBoolean(options.getOrDefault(DATE_AS_TIME_STAMP, Boolean.toString(false))); + boolean doNotMapColumnFamily = Boolean.parseBoolean(options.getOrDefault(DO_NOT_MAP_COLUMN_FAMILY, Boolean.toString(false))); Properties overriddenProps = extractPhoenixHBaseConfFromOptions(options); if (tenant != null) { overriddenProps.put(PhoenixRuntime.TENANT_ID_ATTRIB, tenant); @@ -74,14 +81,13 @@ public StructType inferSchema(CaseInsensitiveStringMap options){ * Sets the schema using all the table columns before any column pruning has been done */ try (Connection conn = DriverManager.getConnection(jdbcUrl, overriddenProps)) { - List columnInfos = PhoenixRuntime.generateColumnInfo(conn, tableName, null); + List columnInfos = generateColumnInfo(conn, tableName); Seq columnInfoSeq = JavaConverters.asScalaIteratorConverter(columnInfos.iterator()).asScala().toSeq(); - schema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoSeq, dateAsTimestamp, doNotMapColumnFamily); - } - catch (SQLException e) { + StructType schema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoSeq, dateAsTimestamp, doNotMapColumnFamily); + return schema; + } catch (SQLException e) { throw new RuntimeException(e); } - return schema; } public static String getJdbcUrlFromOptions(Map options) { @@ -111,8 +117,7 @@ public static String getJdbcUrlFromOptions(Map options) { } @Override - public Table getTable( StructType schema, Transform[] transforms, Map properties) - { + public Table getTable(StructType schema, Transform[] transforms, Map properties) { return new PhoenixTable(schema, properties); } @@ -122,10 +127,11 @@ public Table getTable( StructType schema, Transform[] transforms, Map options) { + public static Properties extractPhoenixHBaseConfFromOptions(final Map options) { Properties confToSet = new Properties(); if (options != null) { String phoenixConfigs = options.get(PHOENIX_CONFIGS); @@ -153,4 +159,34 @@ public static Properties extractPhoenixHBaseConfFromOptions(final Map parameters) { + + return new PhoenixSparkSqlRelation( + sqlContext.sparkSession(), + inferSchema(new CaseInsensitiveStringMap(JavaConverters.mapAsJavaMap(parameters))), + parameters); + } + + //TODO Method PhoenixRuntime.generateColumnInfo skip only salt column, add skip tenant_id column. + private List generateColumnInfo(Connection conn, String tableName) throws SQLException { + List columnInfos = new ArrayList<>(); + PTable table = PhoenixRuntime.getTable(conn, SchemaUtil.normalizeFullTableName(tableName)); + int startOffset = 0; + + if(table.getTenantId()!=null) { + startOffset++; + } + if(table.getBucketNum()!=null){ + startOffset++; + } + + for (int offset = startOffset; offset < table.getColumns().size(); offset++) { + PColumn column = table.getColumns().get(offset); + columnInfos.add(PhoenixRuntime.getColumnInfo(column)); + } + return columnInfos; + } + } diff --git a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/PhoenixTable.java b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/PhoenixTable.java index fdd42171..685c753a 100644 --- a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/PhoenixTable.java +++ b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/PhoenixTable.java @@ -37,17 +37,17 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class PhoenixTable implements SupportsRead, SupportsWrite{ +public class PhoenixTable implements SupportsRead, SupportsWrite { - private final Map options; + private final Map options; private final String tableName; private final StructType schema; private static final Set CAPABILITIES = - ImmutableSet.of(BATCH_READ, BATCH_WRITE, ACCEPT_ANY_SCHEMA); + ImmutableSet.of(BATCH_READ, BATCH_WRITE, ACCEPT_ANY_SCHEMA); - public PhoenixTable(StructType schema, Map options) { + public PhoenixTable(StructType schema, Map options) { this.options = options; - this.tableName = options.get("table"); + this.tableName = options.get(PhoenixDataSource.TABLE); this.schema = schema; } diff --git a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/reader/PhoenixScan.java b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/reader/PhoenixScan.java index c72206ee..3efb8703 100644 --- a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/reader/PhoenixScan.java +++ b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/reader/PhoenixScan.java @@ -71,12 +71,12 @@ public class PhoenixScan implements Scan, Batch { this.whereClause = whereClause; this.overriddenProps = extractPhoenixHBaseConfFromOptions(options); this.jdbcUrl = PhoenixDataSource.getJdbcUrlFromOptions(options); - this.tableName = options.get("table"); + this.tableName = options.get(PhoenixDataSource.TABLE); } private void populateOverriddenProperties(){ currentScnValue = options.get(PhoenixConfigurationUtil.CURRENT_SCN_VALUE); - tenantId = options.get(PhoenixConfigurationUtil.MAPREDUCE_TENANT_ID); + tenantId = options.get(PhoenixDataSource.TENANT_ID); // Generate splits based off statistics, or just region splits? splitByStats = options.getBoolean( PhoenixConfigurationUtil.MAPREDUCE_SPLIT_BY_STATS, PhoenixConfigurationUtil.DEFAULT_SPLIT_BY_STATS); diff --git a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/reader/PhoenixScanBuilder.java b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/reader/PhoenixScanBuilder.java index e898a466..0c2fd53d 100644 --- a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/reader/PhoenixScanBuilder.java +++ b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/reader/PhoenixScanBuilder.java @@ -60,7 +60,8 @@ public Filter[] pushedFilters() { @Override public void pruneColumns(StructType requiredSchema) { - this.schema = requiredSchema; + if (requiredSchema.fields() != null && requiredSchema.fields().length != 0) + this.schema = requiredSchema; } @VisibleForTesting diff --git a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/writer/PhoenixBatchWrite.java b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/writer/PhoenixBatchWrite.java index 5115f735..ebf4f5bb 100644 --- a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/writer/PhoenixBatchWrite.java +++ b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/writer/PhoenixBatchWrite.java @@ -19,7 +19,6 @@ import org.apache.phoenix.spark.sql.connector.PhoenixDataSource; import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.phoenix.util.PhoenixRuntime; import org.apache.spark.sql.connector.write.BatchWrite; import org.apache.spark.sql.connector.write.DataWriterFactory; import org.apache.spark.sql.connector.write.LogicalWriteInfo; @@ -59,9 +58,9 @@ public void abort(WriterCommitMessage[] messages) { private PhoenixDataSourceWriteOptions createPhoenixDataSourceWriteOptions(Map options, StructType schema) { String scn = options.get(CURRENT_SCN_VALUE); - String tenantId = options.get(PhoenixRuntime.TENANT_ID_ATTRIB); + String tenantId = options.get(PhoenixDataSource.TENANT_ID); String jdbcUrl = PhoenixDataSource.getJdbcUrlFromOptions(options); - String tableName = options.get("table"); + String tableName = options.get(PhoenixDataSource.TABLE); boolean skipNormalizingIdentifier = Boolean.parseBoolean(options.getOrDefault(SKIP_NORMALIZING_IDENTIFIER, Boolean.toString(false))); return new PhoenixDataSourceWriteOptions.Builder() .setTableName(tableName) diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala index dbd7b04c..2436122a 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala @@ -14,8 +14,7 @@ package org.apache.phoenix.spark import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hbase.{HBaseConfiguration, HConstants} -import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver +import org.apache.hadoop.hbase.HConstants import org.apache.phoenix.mapreduce.util.{ColumnInfoToStringEncoderDecoder, PhoenixConfigurationUtil} import org.apache.phoenix.query.HBaseFactoryProvider import org.apache.phoenix.util.{ColumnInfo, PhoenixRuntime} diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala index 77c172ac..dd87a101 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala @@ -17,14 +17,12 @@ */ package org.apache.phoenix.spark -import java.sql.Date -import java.sql.Timestamp -import java.text.Format - -import org.apache.phoenix.util.{DateUtil, SchemaUtil} import org.apache.phoenix.util.StringUtil.escapeStringConstant -import org.apache.spark.sql.sources.{And, EqualTo, Filter, GreaterThan, GreaterThanOrEqual, In, -IsNotNull, IsNull, LessThan, LessThanOrEqual, Not, Or, StringContains, StringEndsWith, StringStartsWith} +import org.apache.phoenix.util.{DateUtil, SchemaUtil} +import org.apache.spark.sql.sources._ + +import java.sql.{Date, Timestamp} +import java.text.Format class FilterExpressionCompiler() { diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala index 89d808de..c0af5c4a 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala @@ -14,7 +14,6 @@ package org.apache.phoenix.spark import java.sql.DriverManager - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hbase.HConstants import org.apache.hadoop.io.NullWritable diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala index aacd4600..2edf5463 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala @@ -19,7 +19,7 @@ package org.apache.phoenix.spark import org.apache.hadoop.conf.Configuration import org.apache.spark.rdd.RDD -import org.apache.spark.sql.sources.{BaseRelation, PrunedFilteredScan, Filter} +import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Row, SQLContext} diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala index 19535bfc..8f866136 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala @@ -18,18 +18,9 @@ package org.apache.phoenix.spark import org.apache.phoenix.query.QueryConstants -import org.apache.phoenix.schema.types.{PBinary, PBinaryArray, PBoolean, PBooleanArray, PChar, -PCharArray, PDate, PDateArray, PDecimal, PDecimalArray, PDouble, PDoubleArray, PFloat, PFloatArray, -PInteger, PIntegerArray, PLong, PLongArray, PSmallint, PSmallintArray, PTime, PTimeArray, -PTimestamp, PTimestampArray, PTinyint, PTinyintArray, PUnsignedDate, PUnsignedDateArray, -PUnsignedDouble, PUnsignedDoubleArray, PUnsignedFloat, PUnsignedFloatArray, PUnsignedInt, -PUnsignedIntArray, PUnsignedLong, PUnsignedLongArray, PUnsignedSmallint, PUnsignedSmallintArray, -PUnsignedTime, PUnsignedTimeArray, PUnsignedTimestamp, PUnsignedTimestampArray, PUnsignedTinyint, -PUnsignedTinyintArray, PVarbinary, PVarbinaryArray, PVarchar, PVarcharArray} +import org.apache.phoenix.schema.types._ import org.apache.phoenix.util.{ColumnInfo, SchemaUtil} -import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteType, DataType, DateType, -DecimalType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructField, -StructType, TimestampType} +import org.apache.spark.sql.types._ object SparkSchemaUtil { @@ -42,7 +33,7 @@ object SparkSchemaUtil { new StructType(structFields.toArray) } - private def normalizeColumnName(columnName: String, doNotMapColumnFamily: Boolean ) = { + private def normalizeColumnName(columnName: String, doNotMapColumnFamily: Boolean) = { val unescapedColumnName = SchemaUtil.getUnEscapedFullColumnName(columnName) var normalizedColumnName = "" if (unescapedColumnName.indexOf(QueryConstants.NAME_SEPARATOR) < 0) { diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/PhoenixSparkSqlRelation.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/PhoenixSparkSqlRelation.scala new file mode 100644 index 00000000..2b8e074c --- /dev/null +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/PhoenixSparkSqlRelation.scala @@ -0,0 +1,58 @@ +package org.apache.phoenix.spark.sql.connector + +import org.apache.phoenix.spark.sql.connector.reader.PhoenixScanBuilder +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.execution.datasources.v2.DataSourceRDD +import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation, PrunedFilteredScan} +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql._ + +import scala.collection.JavaConverters._ + +case class PhoenixSparkSqlRelation( + @transient sparkSession: SparkSession, + schema: StructType, + params: Map[String, String] + ) extends BaseRelation with PrunedFilteredScan with InsertableRelation { + override def sqlContext: SQLContext = sparkSession.sqlContext + + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { + val requiredSchema = StructType(requiredColumns.flatMap(c => schema.fields.find(_.name == c))) + val scanBuilder: PhoenixScanBuilder = new PhoenixScanBuilder(requiredSchema, new CaseInsensitiveStringMap(params.asJava)) + scanBuilder.pushFilters(filters) + val batch = scanBuilder.build().toBatch + val rdd = new DataSourceRDD( + sqlContext.sparkContext, + batch.planInputPartitions(), + batch.createReaderFactory(), + false, + Map.empty + ) + rdd.map(ir => { + val data = requiredSchema.zipWithIndex.map { + case (structField, ordinal) => + structField.dataType match { + case StringType => ir.getString(ordinal) + case _ => ir.get(ordinal, structField.dataType) + } + } + new GenericRowWithSchema(data.toArray, requiredSchema) + }) + } + + override def insert(data: DataFrame, overwrite: Boolean): Unit = { + data + .write + .format("phoenix") + .option(PhoenixDataSource.TABLE, params(PhoenixDataSource.TABLE)) + .option(PhoenixDataSource.JDBC_URL, PhoenixDataSource.getJdbcUrlFromOptions(params.asJava)) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER, + params.getOrElse(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER, "false")) + .option(PhoenixDataSource.TENANT_ID,params.getOrElse(PhoenixDataSource.TENANT_ID,null)) + .mode(SaveMode.Append) + .save() + } + +} diff --git a/phoenix5-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/SparkJdbcUtil.scala b/phoenix5-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/SparkJdbcUtil.scala index 4e11eb53..59bf8c67 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/SparkJdbcUtil.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/SparkJdbcUtil.scala @@ -24,7 +24,6 @@ import java.util.Locale import org.apache.spark.executor.InputMetrics import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.catalyst.util.{DateTimeUtils, GenericArrayData} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._