diff --git a/phoenix5-spark/README.md b/phoenix5-spark/README.md index 3542a19d..be8da1e6 100644 --- a/phoenix5-spark/README.md +++ b/phoenix5-spark/README.md @@ -18,6 +18,18 @@ 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` | +| zrUrl | empty | R/W | List of zookeeper hosts. Deprecated, use `jdbcUrl` instead | +| jdbcUrl | empty | R/W | jdbc url connection to database as `jdbc:phoenix:zkHost:zkport` | +| 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,7 +57,7 @@ val spark = SparkSession val df = spark.sqlContext .read .format("phoenix") - .options(Map("table" -> "TABLE1")) + .options(Map("table" -> "TABLE1", "jdbcUrl" -> "jdbc:phoenix:zkHost:zkport")) .load df.filter(df("COL1") === "test_row_1" && df("ID") === 1L) @@ -65,21 +77,67 @@ 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") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .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(); + } +} +``` + +### 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', 'jdbcUrl' 'jdbc:phoenix:zkHost:zkport')") + +val df = spark.sql(s"SELECT ID FROM $sqlTableName where COL1='test_row_1'") + +df.show + +``` +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', 'jdbcUrl' 'jdbc:phoenix:zkHost:zkport')"); + + + df = spark.sql("SELECT * FROM TABLE1 WHERE COL1='test_row_1' AND ID=1L"); df.show(); - jsc.stop(); } } ``` @@ -89,7 +147,7 @@ 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 +`phoenix` for DataSourceV2 and must also pass in a `table` and `jdbcUrl` parameter to specify which table and server to persist the DataFrame to. The column names are derived from the DataFrame's schema field names, and must match the Phoenix column names. @@ -118,14 +176,16 @@ val spark = SparkSession val df = spark.sqlContext .read .format("phoenix") - .options(Map("table" -> "INPUT_TABLE")) + .option("table", "INPUT_TABLE") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .load // Save to OUTPUT_TABLE df.write .format("phoenix") - .mode(SaveMode.Overwrite) - .options(Map("table" -> "OUTPUT_TABLE")) + .mode(SaveMode.Append) + .option("table", "OUTPUT_TABLE") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .save() ``` Java example: @@ -141,15 +201,15 @@ 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") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .load(); // Save to OUTPUT_TABLE @@ -157,8 +217,8 @@ public class PhoenixSparkWriteFromInputTable { .format("phoenix") .mode(SaveMode.Overwrite) .option("table", "OUTPUT_TABLE") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .save(); - jsc.stop(); } } ``` @@ -166,7 +226,7 @@ public class PhoenixSparkWriteFromInputTable { ### 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. +`jdbcUrl` parameters indicating which table and server 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,11 +260,12 @@ 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") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .mode(SaveMode.Overwrite) .save() ``` @@ -230,10 +291,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 +307,13 @@ 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") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .save(); - - jsc.stop(); } } ``` @@ -269,14 +326,10 @@ the deprected `zkUrl` parameter for backwards compatibility purposes. If neither 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"` +- DataSourceV1 is no longer supported as of `connectors-6.0.0`. +Use `"phoenix"` instead of `"org.apache.phoenix.spark"` datasource - 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` are removed as of `connectors-6.0.0`. - 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: @@ -300,7 +353,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", "jdbcUrl" -> "jdbc:phoenix:zkHost:zkport", "doNotMapColumnFamily" -> "true")) .load; ``` @@ -310,75 +363,3 @@ to executors as a comma-separated list against the key `phoenixConfigs` i.e (Pho - The Data Source API does not support passing custom Phoenix settings in configuration, you must create the DataFrame or RDD directly if you need fine-grained configuration. - No support for aggregate or distinct functions (http://phoenix.apache.org/phoenix_mr.html) - -## Deprecated Usages - -### Load as a DataFrame directly using a Configuration object -```scala -import org.apache.hadoop.conf.Configuration -import org.apache.spark.SparkContext -import org.apache.spark.sql.SQLContext -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 sc = new SparkContext("local", "phoenix-test", sparkConf) -val sqlContext = new SQLContext(sc) - -// Load the columns 'ID' and 'COL1' from TABLE1 as a DataFrame -val df = sqlContext.phoenixTableAsDataFrame( - "TABLE1", Array("ID", "COL1"), conf = configuration -) - -df.show -``` - -### Load as an RDD, using a Zookeeper URL -```scala -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) - -// Load the columns 'ID' and 'COL1' from TABLE1 as an RDD -val rdd: RDD[Map[String, AnyRef]] = sc.phoenixTableAsRDD( - "TABLE1", Seq("ID", "COL1"), zkUrl = Some("phoenix-server:2181") -) - -rdd.count() - -val firstId = rdd.first()("ID").asInstanceOf[Long] -val firstCol = rdd.first()("COL1").asInstanceOf[String] -``` - -### Saving RDDs to Phoenix - -`saveToPhoenix` is an implicit method on RDD[Product], or an RDD of Tuples. The data types must -correspond to the Java types Phoenix supports (http://phoenix.apache.org/language/datatypes.html) - -Given a Phoenix table with the following DDL: - -```sql -CREATE TABLE OUTPUT_TEST_TABLE (id BIGINT NOT NULL PRIMARY KEY, col1 VARCHAR, col2 INTEGER); -``` - -```scala -import org.apache.spark.SparkContext -import org.apache.phoenix.spark._ - -val sc = new SparkContext("local", "phoenix-test") -val dataSet = List((1L, "1", 1), (2L, "2", 2), (3L, "3", 3)) - -sc - .parallelize(dataSet) - .saveToPhoenix( - "OUTPUT_TEST_TABLE", - 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..78d59fea 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 @@ -148,7 +148,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 +173,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 +204,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..b406aa27 100644 --- a/phoenix5-spark/src/it/resources/globalSetup.sql +++ b/phoenix5-spark/src/it/resources/globalSetup.sql @@ -60,7 +60,7 @@ UPSERT INTO "small" VALUES ('key1', 'foo', 10000) 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 (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/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala index 1b244034..1a157ebf 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 @@ -21,8 +21,7 @@ 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.spark.sql.SparkSession import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite, Matchers} @@ -64,9 +63,7 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter conf } - lazy val quorumAddress = { - ConfigurationUtil.getZookeeperURL(hbaseConfiguration).get - } + lazy val jdbcUrl = PhoenixSparkITHelper.getUrl // Runs SQL commands located in the file defined in the sqlSource argument // Optional argument tenantId used for running tenant-specific SQL @@ -103,12 +100,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/PhoenixSparkIT.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala index f1a09d64..64c4a7ed 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,14 +13,13 @@ */ 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.spark.datasource.v2.{PhoenixDataSource, PhoenixTestingDataSource} +import org.apache.phoenix.schema.types.{PSmallintArray, PVarchar} +import org.apache.phoenix.spark.datasource.v2.{PhoenixDataSource, PhoenixTestingDataSource, SparkSchemaUtil} import org.apache.phoenix.spark.datasource.v2.reader.PhoenixTestingInputPartitionReader import org.apache.phoenix.spark.datasource.v2.writer.PhoenixTestingDataSourceWriter import org.apache.phoenix.util.{ColumnInfo, SchemaUtil} @@ -47,7 +46,8 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { 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,29 +66,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( @@ -100,7 +77,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { 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() @@ -136,7 +113,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { 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 +163,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 +185,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 +211,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 +221,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 +234,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 +256,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 +272,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 +287,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 +305,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 +321,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 +334,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 +355,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() @@ -428,7 +405,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { 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() @@ -457,7 +434,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 +453,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 +461,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 +469,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 +484,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 +492,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 +523,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 +540,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 +549,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 +572,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 +621,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 +647,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 +678,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 +708,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 +736,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 +766,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,7 +787,7 @@ 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 @@ -829,7 +806,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 +815,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 +834,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,7 +856,7 @@ 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 @@ -891,7 +868,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { 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 @@ -901,12 +878,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { 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 +915,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 +932,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/PhoenixSparkITSaltAndTenantSpecific.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITSaltAndTenantSpecific.scala new file mode 100644 index 00000000..f10dba38 --- /dev/null +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITSaltAndTenantSpecific.scala @@ -0,0 +1,145 @@ +/* + 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.spark.datasource.v2.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 + * + */ +class PhoenixSparkITSaltAndTenantSpecific 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(("testOrg4", "data4"), ("testOrg5", "data5")) + + val sqlTableName = "TENANT_TABLE" + + after { + spark.sql(s"DROP TABLE IF EXISTS $sqlTableName") + } + + /** + * 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(expected: Seq[(String, String)]): Unit = { + // Contains the default data upserted into the tenant-specific table in tenantSetup.sql and the data upserted by tests + + + } + + /** ************** */ + /** Read tests * */ + /** ************** */ + + test("Can read from tenant-specific table as DataFrame") { + 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 result = df.collect() + expected shouldEqual result + } + + + + 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") + + 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.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) + + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual expected + } + + 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/PhoenixSparkITTenantSpecific.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala deleted file mode 100644 index 291ea2a3..00000000 --- a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala +++ /dev/null @@ -1,135 +0,0 @@ -/* - 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.util.PhoenixRuntime -import org.apache.spark.sql.SQLContext - -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 PhoenixSparkITTenantSpecific 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")) - - /** - * 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 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.write - .format("phoenix") - .mode("overwrite") - .option("table", TenantTable) - .option(PhoenixRuntime.TENANT_ID_ATTRIB, TenantId) - .option("zkUrl", PhoenixSparkITHelper.getUrl) - .save() - - 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/PhoenixSparkSqlIT.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala new file mode 100644 index 00000000..28b61b90 --- /dev/null +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala @@ -0,0 +1,95 @@ +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.show() + + 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.show + + 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.show() + + 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.show() + + 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 + } + +} 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..6631bd54 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 @@ -22,6 +22,10 @@ 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 +37,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 +45,7 @@ /** * 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 +53,11 @@ 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) { @@ -56,7 +66,7 @@ public DataSourceReader createReader(DataSourceOptions options) { @Override public Optional createWriter(String writeUUID, StructType schema, - SaveMode mode, DataSourceOptions options) { + SaveMode mode, DataSourceOptions options) { return Optional.of(new PhoenixDataSourceWriter(mode, schema, options)); } @@ -93,6 +103,7 @@ public static String getJdbcUrlFromOptions(final DataSourceOptions options) { * {@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 + * * @param options DataSource options passed in * @return Properties map */ @@ -117,6 +128,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 +140,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..1e0ca13c 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,20 +23,22 @@ 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; +import org.apache.phoenix.spark.datasource.v2.FilterExpressionCompiler; +import org.apache.phoenix.spark.datasource.v2.SparkSchemaUtil; import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource; 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/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/ConfigurationUtil.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala deleted file mode 100644 index dbd7b04c..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala +++ /dev/null @@ -1,109 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.hadoop.hbase.{HBaseConfiguration, HConstants} -import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver -import org.apache.phoenix.mapreduce.util.{ColumnInfoToStringEncoderDecoder, PhoenixConfigurationUtil} -import org.apache.phoenix.query.HBaseFactoryProvider -import org.apache.phoenix.util.{ColumnInfo, PhoenixRuntime} - -import scala.collection.JavaConversions._ - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -object ConfigurationUtil extends Serializable { - - def getOutputConfiguration(tableName: String, columns: Seq[String], zkUrl: Option[String], tenantId: Option[String] = None, conf: Option[Configuration] = None): Configuration = { - - // Create an HBaseConfiguration object from the passed in config, if present - val config = conf match { - case Some(c) => HBaseFactoryProvider.getConfigurationFactory.getConfiguration(c) - case _ => HBaseFactoryProvider.getConfigurationFactory.getConfiguration() - } - - // Set the tenantId in the config if present - tenantId match { - case Some(id) => setTenantId(config, id) - case _ => - } - - // Set the table to save to - PhoenixConfigurationUtil.setOutputTableName(config, tableName) - PhoenixConfigurationUtil.setPhysicalTableName(config, tableName) - // disable property provider evaluation - PhoenixConfigurationUtil.setPropertyPolicyProviderDisabled(config); - - // Infer column names from the DataFrame schema - PhoenixConfigurationUtil.setUpsertColumnNames(config, Array(columns : _*)) - - // Override the Zookeeper URL if present. Throw exception if no address given. - zkUrl match { - case Some(url) => setZookeeperURL(config, url) - case _ => { - if (ConfigurationUtil.getZookeeperURL(config).isEmpty) { - throw new UnsupportedOperationException( - s"One of zkUrl or '${HConstants.ZOOKEEPER_QUORUM}' config property must be provided" - ) - } - } - } - // Return the configuration object - config - } - - def setZookeeperURL(conf: Configuration, zkUrl: String) = { - var zk = zkUrl - if (zk.startsWith("jdbc:phoenix:")) { - zk = zk.substring("jdbc:phoenix:".length) - } - if (zk.startsWith("jdbc:phoenix+zk:")) { - zk = zk.substring("jdbc:phoenix+zk:".length) - } - val escapedUrl = zk.replaceAll("\\\\:","=") - val parts = escapedUrl.split(":") - if (parts.length >= 1 && parts(0).length()>0) - conf.set(HConstants.ZOOKEEPER_QUORUM, parts(0).replaceAll("=", "\\\\:")) - if (parts.length >= 2 && parts(1).length()>0) - conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.parseInt(parts(1).replaceAll("=", "\\\\:"))) - if (parts.length >= 3 && parts(2).length()>0) - conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parts(2).replaceAll("=", "\\\\:")) - } - - def setTenantId(conf: Configuration, tenantId: String) = { - conf.set(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId) - } - - // Return a serializable representation of the columns - def encodeColumns(conf: Configuration) = { - ColumnInfoToStringEncoderDecoder.encode(conf, PhoenixConfigurationUtil.getUpsertColumnMetadataList(conf) - ) - } - - // Decode the columns to a list of ColumnInfo objects - def decodeColumns(conf: Configuration): List[ColumnInfo] = { - ColumnInfoToStringEncoderDecoder.decode(conf).toList - } - - 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(":")) - } - } -} diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala deleted file mode 100644 index 3b0289d2..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala +++ /dev/null @@ -1,79 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.hadoop.io.NullWritable -import org.apache.phoenix.mapreduce.PhoenixOutputFormat -import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil -import org.apache.phoenix.util.SchemaUtil -import org.apache.spark.sql.DataFrame - -import scala.collection.JavaConversions._ - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class DataFrameFunctions(data: DataFrame) extends Serializable { - def saveToPhoenix(parameters: Map[String, String]): Unit = { - saveToPhoenix(parameters("table"), zkUrl = parameters.get("zkUrl"), tenantId = parameters.get("TenantId"), - skipNormalizingIdentifier=parameters.contains("skipNormalizingIdentifier")) - } - def saveToPhoenix(tableName: String, conf: Configuration = new Configuration, - zkUrl: Option[String] = None, tenantId: Option[String] = None, skipNormalizingIdentifier: Boolean = false): Unit = { - - // Retrieve the schema field names and normalize to Phoenix, need to do this outside of mapPartitions - val fieldArray = getFieldArray(skipNormalizingIdentifier, data) - - - // Create a configuration object to use for saving - @transient val outConfig = ConfigurationUtil.getOutputConfiguration(tableName, fieldArray, zkUrl, tenantId, Some(conf)) - - // Retrieve the zookeeper URL - val zkUrlFinal = ConfigurationUtil.getZookeeperURL(outConfig) - - // Map the row objects into PhoenixRecordWritable - val phxRDD = data.rdd.mapPartitions{ rows => - - // Create a within-partition config to retrieve the ColumnInfo list - @transient val partitionConfig = ConfigurationUtil.getOutputConfiguration(tableName, fieldArray, zkUrlFinal, tenantId) - @transient val columns = PhoenixConfigurationUtil.getUpsertColumnMetadataList(partitionConfig).toList - - rows.map { row => - val rec = new PhoenixRecordWritable(columns) - row.toSeq.foreach { e => rec.add(e) } - (null, rec) - } - } - - // Save it - phxRDD.saveAsNewAPIHadoopFile( - Option( - conf.get("mapreduce.output.fileoutputformat.outputdir") - ).getOrElse( - Option(conf.get("mapred.output.dir")).getOrElse("") - ), - classOf[NullWritable], - classOf[PhoenixRecordWritable], - classOf[PhoenixOutputFormat[PhoenixRecordWritable]], - outConfig - ) - } - - def getFieldArray(skipNormalizingIdentifier: Boolean = false, data: DataFrame) = { - if (skipNormalizingIdentifier) { - data.schema.fieldNames.map(x => x) - } else { - data.schema.fieldNames.map(x => SchemaUtil.normalizeIdentifier(x)) - } - } -} diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala deleted file mode 100644 index ccdf5956..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.phoenix.spark - -import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider} -import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class DefaultSource extends RelationProvider with CreatableRelationProvider { - - // Override 'RelationProvider.createRelation', this enables DataFrame.load() - override def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - verifyParameters(parameters) - - new PhoenixRelation( - parameters("table"), - parameters("zkUrl"), - parameters.contains("dateAsTimestamp") - )(sqlContext) - } - - // Override 'CreatableRelationProvider.createRelation', this enables DataFrame.save() - override def createRelation(sqlContext: SQLContext, mode: SaveMode, - parameters: Map[String, String], data: DataFrame): BaseRelation = { - - if (!mode.equals(SaveMode.Overwrite)) { - throw new Exception("SaveMode other than SaveMode.OverWrite is not supported") - } - - verifyParameters(parameters) - - // Save the DataFrame to Phoenix - data.saveToPhoenix(parameters) - - // Return a relation of the saved data - createRelation(sqlContext, parameters) - } - - // Ensure the required parameters are present - def verifyParameters(parameters: Map[String, String]): Unit = { - if (parameters.get("table").isEmpty) throw new RuntimeException("No Phoenix 'table' option defined") - if (parameters.get("zkUrl").isEmpty) throw new RuntimeException("No Phoenix 'zkUrl' option defined") - } - -} 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 deleted file mode 100644 index 89d808de..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala +++ /dev/null @@ -1,149 +0,0 @@ -/* - 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 java.sql.DriverManager - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hbase.HConstants -import org.apache.hadoop.io.NullWritable -import org.apache.phoenix.jdbc.PhoenixDriver -import org.apache.phoenix.mapreduce.PhoenixInputFormat -import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil -import org.apache.phoenix.query.HBaseFactoryProvider -import org.apache.spark.{Partition, SparkContext, TaskContext} -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} - -import scala.collection.JavaConverters._ - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String], - predicate: Option[String] = None, - zkUrl: Option[String] = None, - @transient conf: Configuration, dateAsTimestamp: Boolean = false, - tenantId: Option[String] = None - ) - extends RDD[PhoenixRecordWritable](sc, Nil) { - - // Make sure to register the Phoenix driver - DriverManager.registerDriver(new PhoenixDriver) - - @transient lazy val phoenixConf = { - getPhoenixConfiguration - } - - val phoenixRDD = sc.newAPIHadoopRDD(phoenixConf, - classOf[PhoenixInputFormat[PhoenixRecordWritable]], - classOf[NullWritable], - classOf[PhoenixRecordWritable]) - - override protected def getPartitions: Array[Partition] = { - phoenixRDD.partitions - } - - override protected def getPreferredLocations(split: Partition): Seq[String] = { - phoenixRDD.preferredLocations(split) - } - - @DeveloperApi - override def compute(split: Partition, context: TaskContext) = { - phoenixRDD.compute(split, context).map(r => r._2) - } - - def printPhoenixConfig(conf: Configuration): Unit = { - for (mapEntry <- conf.iterator().asScala) { - val k = mapEntry.getKey - val v = mapEntry.getValue - - if (k.startsWith("phoenix")) { - println(s"$k = $v") - } - } - } - - def getPhoenixConfiguration: Configuration = { - - val config = HBaseFactoryProvider.getConfigurationFactory.getConfiguration(conf); - - PhoenixConfigurationUtil.setInputClass(config, classOf[PhoenixRecordWritable]) - PhoenixConfigurationUtil.setInputTableName(config, table) - PhoenixConfigurationUtil.setPropertyPolicyProviderDisabled(config); - - if(!columns.isEmpty) { - PhoenixConfigurationUtil.setSelectColumnNames(config, columns.toArray) - } - - if(predicate.isDefined) { - PhoenixConfigurationUtil.setInputTableConditions(config, predicate.get) - } - - // Override the Zookeeper URL if present. Throw exception if no address given. - zkUrl match { - case Some(url) => ConfigurationUtil.setZookeeperURL(config, url) - case _ => { - if(ConfigurationUtil.getZookeeperURL(config).isEmpty) { - throw new UnsupportedOperationException( - s"One of zkUrl or '${HConstants.ZOOKEEPER_QUORUM}' config property must be provided" - ) - } - } - } - - tenantId match { - case Some(tid) => ConfigurationUtil.setTenantId(config, tid) - case _ => - } - - config - } - - // Convert our PhoenixRDD to a DataFrame - def toDataFrame(sqlContext: SQLContext): DataFrame = { - val columnInfoList = PhoenixConfigurationUtil - .getSelectColumnMetadataList(new Configuration(phoenixConf)) - .asScala - - // Keep track of the sql type and column names. - val columns: Seq[(String, Int)] = columnInfoList.map(ci => { - (ci.getDisplayName, ci.getSqlType) - }) - - - // Lookup the Spark catalyst types from the Phoenix schema - val structType = SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoList, dateAsTimestamp) - - // Create the data frame from the converted Spark schema - sqlContext.createDataFrame(map(pr => { - - // Create a sequence of column data - val rowSeq = columns.map { case (name, sqlType) => - val res = pr.resultMap(name) - // Special handling for data types - if (dateAsTimestamp && (sqlType == 91 || sqlType == 19) && res!=null) { // 91 is the defined type for Date and 19 for UNSIGNED_DATE - new java.sql.Timestamp(res.asInstanceOf[java.sql.Date].getTime) - } else if ((sqlType == 92 || sqlType == 18) && res!=null) { // 92 is the defined type for Time and 18 for UNSIGNED_TIME - new java.sql.Timestamp(res.asInstanceOf[java.sql.Time].getTime) - } else { - res - } - } - - // Create a Spark Row from the sequence - Row.fromSeq(rowSeq) - }), structType) - } - -} diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala deleted file mode 100644 index 66c347e3..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala +++ /dev/null @@ -1,115 +0,0 @@ -/* - 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 java.sql.{PreparedStatement, ResultSet} -import org.apache.hadoop.mapreduce.lib.db.DBWritable -import org.apache.phoenix.schema.types.{PBinary, PDataType, PDate, PVarbinary, PhoenixArray} -import org.apache.phoenix.util.ColumnInfo -import org.joda.time.DateTime -import scala.collection.{mutable, immutable} - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class PhoenixRecordWritable(columnMetaDataList: List[ColumnInfo]) extends DBWritable { - val upsertValues = mutable.ArrayBuffer[Any]() - val resultMap = mutable.Map[String, AnyRef]() - - def result : immutable.Map[String, AnyRef] = { - resultMap.toMap - } - - override def write(statement: PreparedStatement): Unit = { - // Make sure we at least line up in size - if(upsertValues.length != columnMetaDataList.length) { - throw new UnsupportedOperationException( - s"Upsert values ($upsertValues) do not match the specified columns (columnMetaDataList)" - ) - } - - // Correlate each value (v) to a column type (c) and an index (i) - upsertValues.zip(columnMetaDataList).zipWithIndex.foreach { - case ((v, c), i) => { - if (v != null) { - - // Both Java and Joda dates used to work in 4.2.3, but now they must be java.sql.Date - // Can override any other types here as needed - val (finalObj, finalType) = v match { - case dt: DateTime => (new java.sql.Date(dt.getMillis), PDate.INSTANCE) - case d: java.util.Date => (new java.sql.Date(d.getTime), PDate.INSTANCE) - case _ => (v, c.getPDataType) - } - - - // Helper method to create an SQL array for a specific PDatatype, and set it on the statement - def setArrayInStatement(obj: Array[AnyRef]): Unit = { - // Create a java.sql.Array, need to lookup the base sql type name - val sqlArray = statement.getConnection.createArrayOf( - PDataType.arrayBaseType(finalType).getSqlTypeName, - obj - ) - statement.setArray(i + 1, sqlArray) - } - - // Determine whether to save as an array or object - (finalObj, finalType) match { - case (obj: Array[AnyRef], _) => setArrayInStatement(obj) - case (obj: mutable.ArrayBuffer[AnyVal], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef]).toArray) - case (obj: mutable.ArrayBuffer[AnyRef], _) => setArrayInStatement(obj.toArray) - case (obj: mutable.WrappedArray[AnyVal], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef]).toArray) - case (obj: mutable.WrappedArray[AnyRef], _) => setArrayInStatement(obj.toArray) - case (obj: Array[Int], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Long], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Char], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Short], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Float], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Double], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - // PVarbinary and PBinary come in as Array[Byte] but they're SQL objects - case (obj: Array[Byte], _ : PVarbinary) => statement.setObject(i + 1, obj) - case (obj: Array[Byte], _ : PBinary) => statement.setObject(i + 1, obj) - // Otherwise set as array type - case (obj: Array[Byte], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case _ => statement.setObject(i + 1, finalObj) - } - } else { - statement.setNull(i + 1, c.getSqlType) - } - } - } - } - - override def readFields(resultSet: ResultSet): Unit = { - val metadata = resultSet.getMetaData - for(i <- 1 to metadata.getColumnCount) { - - // Return the contents of a PhoenixArray, if necessary - val value = resultSet.getObject(i) match { - case x: PhoenixArray => x.getArray - case y => y - } - - // Put a (ColumnLabel -> value) entry in the result map - resultMap(metadata.getColumnLabel(i)) = value - } - } - - def add(value: Any): Unit = { - upsertValues.append(value) - } - - // Empty constructor for MapReduce - def this() = { - this(List[ColumnInfo]()) - } - -} 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 deleted file mode 100644 index aacd4600..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.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.types.StructType -import org.apache.spark.sql.{Row, SQLContext} - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Boolean = false)(@transient val sqlContext: SQLContext) - extends BaseRelation with PrunedFilteredScan { - - /* - This is the buildScan() implementing Spark's PrunedFilteredScan. - Spark SQL queries with columns or predicates specified will be pushed down - to us here, and we can pass that on to Phoenix. According to the docs, this - is an optimization, and the filtering/pruning will be re-evaluated again, - but this prevents having to load the whole table into Spark first. - */ - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { - val(pushedFilters, _, _) = new FilterExpressionCompiler().pushFilters(filters) - new PhoenixRDD( - sqlContext.sparkContext, - tableName, - requiredColumns, - Some(pushedFilters), - Some(zkUrl), - new Configuration(), - dateAsTimestamp - ).toDataFrame(sqlContext).rdd - } - - // Required by BaseRelation, this will return the full schema for a table - override def schema: StructType = { - new PhoenixRDD( - sqlContext.sparkContext, - tableName, - Seq(), - None, - Some(zkUrl), - new Configuration(), - dateAsTimestamp - ).toDataFrame(sqlContext).schema - } - - - override def unhandledFilters(filters: Array[Filter]): Array[Filter] = { - val (_, unhandledFilters, _) = new FilterExpressionCompiler().pushFilters(filters) - unhandledFilters - } - -} diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala deleted file mode 100644 index b0735218..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.hadoop.io.NullWritable -import org.apache.phoenix.mapreduce.PhoenixOutputFormat -import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil -import org.apache.spark.rdd.RDD - -import scala.collection.JavaConversions._ - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class ProductRDDFunctions[A <: Product](data: RDD[A]) extends Serializable { - - def saveToPhoenix(tableName: String, cols: Seq[String], - conf: Configuration = new Configuration, zkUrl: Option[String] = None, tenantId: Option[String] = None) - : Unit = { - - // Create a configuration object to use for saving - @transient val outConfig = ConfigurationUtil.getOutputConfiguration(tableName, cols, zkUrl, tenantId, Some(conf)) - - // Retrieve the zookeeper URL - val zkUrlFinal = ConfigurationUtil.getZookeeperURL(outConfig) - - // Map the row objects into PhoenixRecordWritable - val phxRDD = data.mapPartitions{ rows => - - // Create a within-partition config to retrieve the ColumnInfo list - @transient val partitionConfig = ConfigurationUtil.getOutputConfiguration(tableName, cols, zkUrlFinal, tenantId) - @transient val columns = PhoenixConfigurationUtil.getUpsertColumnMetadataList(partitionConfig).toList - - rows.map { row => - val rec = new PhoenixRecordWritable(columns) - row.productIterator.foreach { e => rec.add(e) } - (null, rec) - } - } - - // Save it - phxRDD.saveAsNewAPIHadoopFile( - Option( - conf.get("mapreduce.output.fileoutputformat.outputdir") - ).getOrElse( - Option(conf.get("mapred.output.dir")).getOrElse("") - ), - classOf[NullWritable], - classOf[PhoenixRecordWritable], - classOf[PhoenixOutputFormat[PhoenixRecordWritable]], - outConfig - ) - } -} \ No newline at end of file diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala deleted file mode 100644 index 1b377abe..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class SparkContextFunctions(@transient val sc: SparkContext) extends Serializable { - - /* - This will return an RDD of Map[String, AnyRef], where the String key corresponds to the column - name and the AnyRef value will be a java.sql type as returned by Phoenix - - 'table' is the corresponding Phoenix table - 'columns' is a sequence of of columns to query - 'predicate' is a set of statements to go after a WHERE clause, e.g. "TID = 123" - 'zkUrl' is an optional Zookeeper URL to use to connect to Phoenix - 'conf' is a Hadoop Configuration object. If zkUrl is not set, the "hbase.zookeeper.quorum" - property will be used - */ - - def phoenixTableAsRDD(table: String, columns: Seq[String], predicate: Option[String] = None, - zkUrl: Option[String] = None, tenantId: Option[String] = None, conf: Configuration = new Configuration()) - : RDD[Map[String, AnyRef]] = { - - // Create a PhoenixRDD, but only return the serializable 'result' map - new PhoenixRDD(sc, table, columns, predicate, zkUrl, conf, tenantId = tenantId).map(_.result) - } -} \ No newline at end of file diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala deleted file mode 100644 index f9154adf..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.spark.sql.{DataFrame, SQLContext} - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class SparkSqlContextFunctions(@transient val sqlContext: SQLContext) extends Serializable { - - /* - This will return a Spark DataFrame, with Phoenix types converted Spark SQL catalyst types - - 'table' is the corresponding Phoenix table - 'columns' is a sequence of of columns to query - 'predicate' is a set of statements to go after a WHERE clause, e.g. "TID = 123" - 'zkUrl' is an optional Zookeeper URL to use to connect to Phoenix - 'conf' is a Hadoop Configuration object. If zkUrl is not set, the "hbase.zookeeper.quorum" - property will be used - */ - def phoenixTableAsDataFrame(table: String, columns: Seq[String], - predicate: Option[String] = None, - zkUrl: Option[String] = None, - tenantId: Option[String] = None, - conf: Configuration = new Configuration): DataFrame = { - - // Create the PhoenixRDD and convert it to a DataFrame - new PhoenixRDD(sqlContext.sparkContext, table, columns, predicate, zkUrl, conf, tenantId = tenantId) - .toDataFrame(sqlContext) - } -} \ No newline at end of file diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/FilterExpressionCompiler.scala similarity index 95% rename from phoenix5-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala rename to phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/FilterExpressionCompiler.scala index 111f021c..2c70698c 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/FilterExpressionCompiler.scala @@ -15,16 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.phoenix.spark +package org.apache.phoenix.spark.datasource.v2 -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/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/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/SparkSchemaUtil.scala similarity index 84% rename from phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala rename to phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/SparkSchemaUtil.scala index 19535bfc..d2a912a9 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/datasource/v2/SparkSchemaUtil.scala @@ -15,21 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.phoenix.spark +package org.apache.phoenix.spark.datasource.v2 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/package.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/package.scala deleted file mode 100644 index 3fed79ea..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/package.scala +++ /dev/null @@ -1,36 +0,0 @@ -/* - 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 - -import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext} - -package object spark { - implicit def toProductRDDFunctions[A <: Product](rdd: RDD[A]): ProductRDDFunctions[A] = { - new ProductRDDFunctions[A](rdd) - } - - implicit def toSparkContextFunctions(sc: SparkContext): SparkContextFunctions = { - new SparkContextFunctions(sc) - } - - implicit def toSparkSqlContextFunctions(sqlContext: SQLContext): SparkSqlContextFunctions = { - new SparkSqlContextFunctions(sqlContext) - } - - implicit def toDataFrameFunctions(data: DataFrame): DataFrameFunctions = { - new DataFrameFunctions(data) - } -} diff --git a/phoenix5-spark3/README.md b/phoenix5-spark3/README.md index 8954e168..239a7584 100644 --- a/phoenix5-spark3/README.md +++ b/phoenix5-spark3/README.md @@ -28,6 +28,18 @@ 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` | +| zrUrl | empty | R/W | List of zookeeper hosts. Deprecated, use `jdbcUrl` instead | +| jdbcUrl | empty | R/W | jdbc url connection to database as `jdbc:phoenix:zkHost:zkport` | +| 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,7 +67,7 @@ val spark = SparkSession val df = spark.sqlContext .read .format("phoenix") - .options(Map("table" -> "TABLE1")) + .options(Map("table" -> "TABLE1", "jdbcUrl" -> "jdbc:phoenix:zkHost:zkport")) .load df.filter(df("COL1") === "test_row_1" && df("ID") === 1L) @@ -75,21 +87,67 @@ 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") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .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(); + } +} +``` + +### 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', 'jdbcUrl' 'jdbc:phoenix:zkHost:zkport')") + +val df = spark.sql(s"SELECT ID FROM $sqlTableName where COL1='test_row_1'") + +df.show + +``` +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', 'jdbcUrl' 'jdbc:phoenix:zkHost:zkport')"); + + + df = spark.sql("SELECT * FROM TABLE1 WHERE COL1='test_row_1' AND ID=1L"); df.show(); - jsc.stop(); } } ``` @@ -99,7 +157,7 @@ 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 +`phoenix` for DataSourceV2 and must also pass in a `table` and `jdbcUrl` parameter to specify which table and server to persist the DataFrame to. The column names are derived from the DataFrame's schema field names, and must match the Phoenix column names. @@ -128,14 +186,16 @@ val spark = SparkSession val df = spark.sqlContext .read .format("phoenix") - .options(Map("table" -> "INPUT_TABLE")) + .option("table", "INPUT_TABLE") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .load // Save to OUTPUT_TABLE df.write .format("phoenix") .mode(SaveMode.Append) - .options(Map("table" -> "OUTPUT_TABLE")) + .option("table", "OUTPUT_TABLE") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .save() ``` Java example: @@ -152,14 +212,14 @@ 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") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .load(); // Save to OUTPUT_TABLE @@ -167,8 +227,8 @@ public class PhoenixSparkWriteFromInputTable { .format("phoenix") .mode(SaveMode.Append) .option("table", "OUTPUT_TABLE") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .save(); - jsc.stop(); } } ``` @@ -176,7 +236,7 @@ public class PhoenixSparkWriteFromInputTable { ### 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. +`jdbcUrl` parameters indicating which table and server 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,11 +270,12 @@ 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") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .mode(SaveMode.Append) .save() ``` @@ -240,10 +301,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 +317,13 @@ 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") + .option("jdbcUrl", "jdbc:phoenix:zkHost:zkport") .save(); - - jsc.stop(); } } ``` @@ -279,14 +336,10 @@ the deprected `zkUrl` parameter for backwards compatibility purposes. If neither 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"` +- DataSourceV1 is no longer supported as of `connectors-6.0.0`. +Use `"phoenix"` instead of `"org.apache.phoenix.spark"` datasource - 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` are removed as of `connectors-6.0.0`. - 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: @@ -309,87 +362,13 @@ 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", "jdbcUrl" -> "jdbc:phoenix:zkHost:zkport", "doNotMapColumnFamily" -> "true")) .load; ``` + ## Limitations - Basic support for column and predicate pushdown using the Data Source API - The Data Source API does not support passing custom Phoenix settings in configuration, you must create the DataFrame or RDD directly if you need fine-grained configuration. - No support for aggregate or distinct functions (http://phoenix.apache.org/phoenix_mr.html) - -## Deprecated Usages - -### 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._ - -val configuration = new Configuration() -// Can set Phoenix-specific settings, requires 'hbase.zookeeper.quorum' - -val sparkConf = new SparkConf().set("spark.ui.showConsoleProgress", "false") -val sc = new SparkContext("local", "phoenix-test", sparkConf) -val sqlContext = new SQLContext(sc) - -// Load the columns 'ID' and 'COL1' from TABLE1 as a DataFrame -val df = sqlContext.phoenixTableAsDataFrame( - "TABLE1", Array("ID", "COL1"), conf = configuration -) - -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 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( - "TABLE1", Seq("ID", "COL1"), zkUrl = Some("phoenix-server:2181") -) - -rdd.count() - -val firstId = rdd.first()("ID").asInstanceOf[Long] -val firstCol = rdd.first()("COL1").asInstanceOf[String] -``` - -### Saving RDDs to Phoenix - -`saveToPhoenix` is an implicit method on RDD[Product], or an RDD of Tuples. The data types must -correspond to the Java types Phoenix supports (http://phoenix.apache.org/language/datatypes.html) - -Given a Phoenix table with the following DDL: - -```sql -CREATE TABLE OUTPUT_TEST_TABLE (id BIGINT NOT NULL PRIMARY KEY, col1 VARCHAR, col2 INTEGER); -``` - -```scala -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 sc = new SparkContext("local", "phoenix-test", sparkConf) -val dataSet = List((1L, "1", 1), (2L, "2", 2), (3L, "3", 3)) - -sc - .parallelize(dataSet) - .saveToPhoenix( - "OUTPUT_TEST_TABLE", - 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..849bd0c9 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,8 @@ 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.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 +127,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 +243,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 +293,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 +384,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 +446,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 " @@ -522,9 +515,4 @@ public void testColumnFamily() throws Exception { } } - @Test - @Ignore - public void testOrderByNullable() throws SQLException { - - } } 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..4a22d32e 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 @@ -17,7 +17,6 @@ */ package org.apache.phoenix.spark.sql.connector; -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; @@ -41,19 +40,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..4db41c2c 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') @@ -60,10 +60,11 @@ UPSERT INTO "small" VALUES ('key1', 'foo', 10000) 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 (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/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala index 12e679b4..928f9548 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,18 +13,16 @@ */ 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.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 + // Helper object to access the protected abstract static methods hidden in BaseTest object PhoenixSparkITHelper extends BaseTest { @@ -49,8 +47,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 +62,25 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter conf } - lazy val quorumAddress = { - ConfigurationUtil.getZookeeperURL(hbaseConfiguration).get - } + lazy val jdbcUrl = PhoenixSparkITHelper.getUrl + +// +// 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) } 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..4bd662e6 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,8 +18,8 @@ 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.spark.sql.connector.{PhoenixDataSource, PhoenixTestingDataSource} +import org.apache.phoenix.schema.types.{PSmallintArray, PVarchar} +import org.apache.phoenix.spark.sql.connector.{PhoenixDataSource, PhoenixTestingDataSource, SparkSchemaUtil} import org.apache.phoenix.spark.sql.connector.reader.PhoenixTestPartitionReader import org.apache.phoenix.spark.sql.connector.writer.PhoenixTestBatchWrite import org.apache.phoenix.util.{ColumnInfo, SchemaUtil} @@ -45,8 +45,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 +66,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 +76,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 +112,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 +163,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 +185,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 +211,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 +221,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 +234,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 +256,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 +272,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 +287,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 +305,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 +321,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 +334,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 +356,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 +406,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 +435,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 +454,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 +462,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 +472,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 +487,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 +526,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 +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(PhoenixDataSource.TABLE -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.JDBC_URL -> jdbcUrl)).load() var count = df1.count() @@ -574,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(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.JDBC_URL -> jdbcUrl)) .load() count = df2.count() @@ -597,13 +575,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 +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(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 +650,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 +681,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 +711,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 +739,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 +769,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,7 +790,7 @@ 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 @@ -831,7 +809,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 +818,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 +837,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,7 +859,7 @@ 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 @@ -893,7 +871,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { 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 @@ -903,12 +881,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT { 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 +916,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 +933,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/PhoenixSparkITSaltAndTenantSpecific.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITSaltAndTenantSpecific.scala new file mode 100644 index 00000000..a72162a1 --- /dev/null +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITSaltAndTenantSpecific.scala @@ -0,0 +1,131 @@ +/* + 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.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 + * + */ +class PhoenixSparkITSaltAndTenantSpecific 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(("testOrg4", "data4"), ("testOrg5", "data5")) + + val sqlTableName = "TENANT_TABLE" + + after { + spark.sql(s"DROP TABLE IF EXISTS $sqlTableName") + } + + /** ************** */ + /** Read tests * */ + /** ************** */ + + test("Can read from tenant-specific table as DataFrame") { + 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 result = df.collect() + expected shouldEqual result + } + + 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") + + 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.write + .format("phoenix") + .mode(SaveMode.Append) + .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) + + val results = ListBuffer[(String, String)]() + while (rs.next()) { + results.append((rs.getString(1), rs.getString(2))) + } + stmt.close() + results.toList shouldEqual expected + } + + 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/PhoenixSparkITTenantSpecific.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala deleted file mode 100644 index 7a9b2180..00000000 --- a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala +++ /dev/null @@ -1,153 +0,0 @@ -/* - 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.util.PhoenixRuntime -import org.apache.spark.sql.SQLContext - -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 PhoenixSparkITTenantSpecific 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 - } - - ignore("Can write a DataFrame using 'DataFrame.write' to tenant-specific view - Spark2 sparse columns") { - val sqlContext = spark.sqlContext - import sqlContext.implicits._ - - val df = spark.sparkContext.parallelize(TestDataSet2).toDF(OrgIdCol, TenantOnlyCol); - - df.write - .format("phoenix") - .mode("overwrite") - .option("table", TenantTable) - .option(PhoenixRuntime.TENANT_ID_ATTRIB, TenantId) - .option("zkUrl", PhoenixSparkITHelper.getUrl) - .save() - - verifyResults - } - - test("Can write a DataFrame using 'DataFrame.write' to tenant-specific view - Spark3 all columns") { - val sqlContext = spark.sqlContext - import sqlContext.implicits._ - - val df = spark.sparkContext.parallelize(TestDataSet2).toDF(OrgIdCol, TenantOnlyCol, "TENANT_ID", "GLOBAL_COL1") - - df.write - .format("phoenix") - .mode("append") - .option("table", TenantTable) - .option(PhoenixRuntime.TENANT_ID_ATTRIB, TenantId) - .option("zkUrl", PhoenixSparkITHelper.getUrl) - .save() - - } - - 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/PhoenixSparkSqlIT.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala new file mode 100644 index 00000000..f6f2b9b4 --- /dev/null +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkSqlIT.scala @@ -0,0 +1,90 @@ +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 + } + +} 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..fb916130 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,14 @@ */ package org.apache.phoenix.spark.sql.connector; +import org.apache.phoenix.schema.PColumn; +import org.apache.phoenix.schema.PTable; +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 +39,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 +47,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 +55,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 +80,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 +116,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 +126,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 +158,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..63e3aa49 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 @@ -18,7 +18,7 @@ package org.apache.phoenix.spark.sql.connector.reader; import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.phoenix.spark.FilterExpressionCompiler; +import org.apache.phoenix.spark.sql.connector.FilterExpressionCompiler; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.connector.read.SupportsPushDownFilters; @@ -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 deleted file mode 100644 index dbd7b04c..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala +++ /dev/null @@ -1,109 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.hadoop.hbase.{HBaseConfiguration, HConstants} -import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver -import org.apache.phoenix.mapreduce.util.{ColumnInfoToStringEncoderDecoder, PhoenixConfigurationUtil} -import org.apache.phoenix.query.HBaseFactoryProvider -import org.apache.phoenix.util.{ColumnInfo, PhoenixRuntime} - -import scala.collection.JavaConversions._ - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -object ConfigurationUtil extends Serializable { - - def getOutputConfiguration(tableName: String, columns: Seq[String], zkUrl: Option[String], tenantId: Option[String] = None, conf: Option[Configuration] = None): Configuration = { - - // Create an HBaseConfiguration object from the passed in config, if present - val config = conf match { - case Some(c) => HBaseFactoryProvider.getConfigurationFactory.getConfiguration(c) - case _ => HBaseFactoryProvider.getConfigurationFactory.getConfiguration() - } - - // Set the tenantId in the config if present - tenantId match { - case Some(id) => setTenantId(config, id) - case _ => - } - - // Set the table to save to - PhoenixConfigurationUtil.setOutputTableName(config, tableName) - PhoenixConfigurationUtil.setPhysicalTableName(config, tableName) - // disable property provider evaluation - PhoenixConfigurationUtil.setPropertyPolicyProviderDisabled(config); - - // Infer column names from the DataFrame schema - PhoenixConfigurationUtil.setUpsertColumnNames(config, Array(columns : _*)) - - // Override the Zookeeper URL if present. Throw exception if no address given. - zkUrl match { - case Some(url) => setZookeeperURL(config, url) - case _ => { - if (ConfigurationUtil.getZookeeperURL(config).isEmpty) { - throw new UnsupportedOperationException( - s"One of zkUrl or '${HConstants.ZOOKEEPER_QUORUM}' config property must be provided" - ) - } - } - } - // Return the configuration object - config - } - - def setZookeeperURL(conf: Configuration, zkUrl: String) = { - var zk = zkUrl - if (zk.startsWith("jdbc:phoenix:")) { - zk = zk.substring("jdbc:phoenix:".length) - } - if (zk.startsWith("jdbc:phoenix+zk:")) { - zk = zk.substring("jdbc:phoenix+zk:".length) - } - val escapedUrl = zk.replaceAll("\\\\:","=") - val parts = escapedUrl.split(":") - if (parts.length >= 1 && parts(0).length()>0) - conf.set(HConstants.ZOOKEEPER_QUORUM, parts(0).replaceAll("=", "\\\\:")) - if (parts.length >= 2 && parts(1).length()>0) - conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.parseInt(parts(1).replaceAll("=", "\\\\:"))) - if (parts.length >= 3 && parts(2).length()>0) - conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parts(2).replaceAll("=", "\\\\:")) - } - - def setTenantId(conf: Configuration, tenantId: String) = { - conf.set(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId) - } - - // Return a serializable representation of the columns - def encodeColumns(conf: Configuration) = { - ColumnInfoToStringEncoderDecoder.encode(conf, PhoenixConfigurationUtil.getUpsertColumnMetadataList(conf) - ) - } - - // Decode the columns to a list of ColumnInfo objects - def decodeColumns(conf: Configuration): List[ColumnInfo] = { - ColumnInfoToStringEncoderDecoder.decode(conf).toList - } - - 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(":")) - } - } -} diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala deleted file mode 100644 index 3b0289d2..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala +++ /dev/null @@ -1,79 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.hadoop.io.NullWritable -import org.apache.phoenix.mapreduce.PhoenixOutputFormat -import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil -import org.apache.phoenix.util.SchemaUtil -import org.apache.spark.sql.DataFrame - -import scala.collection.JavaConversions._ - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class DataFrameFunctions(data: DataFrame) extends Serializable { - def saveToPhoenix(parameters: Map[String, String]): Unit = { - saveToPhoenix(parameters("table"), zkUrl = parameters.get("zkUrl"), tenantId = parameters.get("TenantId"), - skipNormalizingIdentifier=parameters.contains("skipNormalizingIdentifier")) - } - def saveToPhoenix(tableName: String, conf: Configuration = new Configuration, - zkUrl: Option[String] = None, tenantId: Option[String] = None, skipNormalizingIdentifier: Boolean = false): Unit = { - - // Retrieve the schema field names and normalize to Phoenix, need to do this outside of mapPartitions - val fieldArray = getFieldArray(skipNormalizingIdentifier, data) - - - // Create a configuration object to use for saving - @transient val outConfig = ConfigurationUtil.getOutputConfiguration(tableName, fieldArray, zkUrl, tenantId, Some(conf)) - - // Retrieve the zookeeper URL - val zkUrlFinal = ConfigurationUtil.getZookeeperURL(outConfig) - - // Map the row objects into PhoenixRecordWritable - val phxRDD = data.rdd.mapPartitions{ rows => - - // Create a within-partition config to retrieve the ColumnInfo list - @transient val partitionConfig = ConfigurationUtil.getOutputConfiguration(tableName, fieldArray, zkUrlFinal, tenantId) - @transient val columns = PhoenixConfigurationUtil.getUpsertColumnMetadataList(partitionConfig).toList - - rows.map { row => - val rec = new PhoenixRecordWritable(columns) - row.toSeq.foreach { e => rec.add(e) } - (null, rec) - } - } - - // Save it - phxRDD.saveAsNewAPIHadoopFile( - Option( - conf.get("mapreduce.output.fileoutputformat.outputdir") - ).getOrElse( - Option(conf.get("mapred.output.dir")).getOrElse("") - ), - classOf[NullWritable], - classOf[PhoenixRecordWritable], - classOf[PhoenixOutputFormat[PhoenixRecordWritable]], - outConfig - ) - } - - def getFieldArray(skipNormalizingIdentifier: Boolean = false, data: DataFrame) = { - if (skipNormalizingIdentifier) { - data.schema.fieldNames.map(x => x) - } else { - data.schema.fieldNames.map(x => SchemaUtil.normalizeIdentifier(x)) - } - } -} diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala deleted file mode 100644 index 7260c1b4..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.phoenix.spark - -import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider} -import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -//This class is used by the new PhoenixDataSource implementation -class DefaultSource extends RelationProvider with CreatableRelationProvider { - - // Override 'RelationProvider.createRelation', this enables DataFrame.load() - override def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - verifyParameters(parameters) - - new PhoenixRelation( - parameters("table"), - parameters("zkUrl"), - parameters.contains("dateAsTimestamp") - )(sqlContext) - } - - // Override 'CreatableRelationProvider.createRelation', this enables DataFrame.save() - override def createRelation(sqlContext: SQLContext, mode: SaveMode, - parameters: Map[String, String], data: DataFrame): BaseRelation = { - - if (!mode.equals(SaveMode.Overwrite)) { - throw new Exception("SaveMode other than SaveMode.OverWrite is not supported") - } - - verifyParameters(parameters) - - // Save the DataFrame to Phoenix - data.saveToPhoenix(parameters) - - // Return a relation of the saved data - createRelation(sqlContext, parameters) - } - - // Ensure the required parameters are present - def verifyParameters(parameters: Map[String, String]): Unit = { - if (parameters.get("table").isEmpty) throw new RuntimeException("No Phoenix 'table' option defined") - if (parameters.get("zkUrl").isEmpty) throw new RuntimeException("No Phoenix 'zkUrl' option defined") - } - -} 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 deleted file mode 100644 index 89d808de..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala +++ /dev/null @@ -1,149 +0,0 @@ -/* - 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 java.sql.DriverManager - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hbase.HConstants -import org.apache.hadoop.io.NullWritable -import org.apache.phoenix.jdbc.PhoenixDriver -import org.apache.phoenix.mapreduce.PhoenixInputFormat -import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil -import org.apache.phoenix.query.HBaseFactoryProvider -import org.apache.spark.{Partition, SparkContext, TaskContext} -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} - -import scala.collection.JavaConverters._ - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String], - predicate: Option[String] = None, - zkUrl: Option[String] = None, - @transient conf: Configuration, dateAsTimestamp: Boolean = false, - tenantId: Option[String] = None - ) - extends RDD[PhoenixRecordWritable](sc, Nil) { - - // Make sure to register the Phoenix driver - DriverManager.registerDriver(new PhoenixDriver) - - @transient lazy val phoenixConf = { - getPhoenixConfiguration - } - - val phoenixRDD = sc.newAPIHadoopRDD(phoenixConf, - classOf[PhoenixInputFormat[PhoenixRecordWritable]], - classOf[NullWritable], - classOf[PhoenixRecordWritable]) - - override protected def getPartitions: Array[Partition] = { - phoenixRDD.partitions - } - - override protected def getPreferredLocations(split: Partition): Seq[String] = { - phoenixRDD.preferredLocations(split) - } - - @DeveloperApi - override def compute(split: Partition, context: TaskContext) = { - phoenixRDD.compute(split, context).map(r => r._2) - } - - def printPhoenixConfig(conf: Configuration): Unit = { - for (mapEntry <- conf.iterator().asScala) { - val k = mapEntry.getKey - val v = mapEntry.getValue - - if (k.startsWith("phoenix")) { - println(s"$k = $v") - } - } - } - - def getPhoenixConfiguration: Configuration = { - - val config = HBaseFactoryProvider.getConfigurationFactory.getConfiguration(conf); - - PhoenixConfigurationUtil.setInputClass(config, classOf[PhoenixRecordWritable]) - PhoenixConfigurationUtil.setInputTableName(config, table) - PhoenixConfigurationUtil.setPropertyPolicyProviderDisabled(config); - - if(!columns.isEmpty) { - PhoenixConfigurationUtil.setSelectColumnNames(config, columns.toArray) - } - - if(predicate.isDefined) { - PhoenixConfigurationUtil.setInputTableConditions(config, predicate.get) - } - - // Override the Zookeeper URL if present. Throw exception if no address given. - zkUrl match { - case Some(url) => ConfigurationUtil.setZookeeperURL(config, url) - case _ => { - if(ConfigurationUtil.getZookeeperURL(config).isEmpty) { - throw new UnsupportedOperationException( - s"One of zkUrl or '${HConstants.ZOOKEEPER_QUORUM}' config property must be provided" - ) - } - } - } - - tenantId match { - case Some(tid) => ConfigurationUtil.setTenantId(config, tid) - case _ => - } - - config - } - - // Convert our PhoenixRDD to a DataFrame - def toDataFrame(sqlContext: SQLContext): DataFrame = { - val columnInfoList = PhoenixConfigurationUtil - .getSelectColumnMetadataList(new Configuration(phoenixConf)) - .asScala - - // Keep track of the sql type and column names. - val columns: Seq[(String, Int)] = columnInfoList.map(ci => { - (ci.getDisplayName, ci.getSqlType) - }) - - - // Lookup the Spark catalyst types from the Phoenix schema - val structType = SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoList, dateAsTimestamp) - - // Create the data frame from the converted Spark schema - sqlContext.createDataFrame(map(pr => { - - // Create a sequence of column data - val rowSeq = columns.map { case (name, sqlType) => - val res = pr.resultMap(name) - // Special handling for data types - if (dateAsTimestamp && (sqlType == 91 || sqlType == 19) && res!=null) { // 91 is the defined type for Date and 19 for UNSIGNED_DATE - new java.sql.Timestamp(res.asInstanceOf[java.sql.Date].getTime) - } else if ((sqlType == 92 || sqlType == 18) && res!=null) { // 92 is the defined type for Time and 18 for UNSIGNED_TIME - new java.sql.Timestamp(res.asInstanceOf[java.sql.Time].getTime) - } else { - res - } - } - - // Create a Spark Row from the sequence - Row.fromSeq(rowSeq) - }), structType) - } - -} diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala deleted file mode 100644 index 66c347e3..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala +++ /dev/null @@ -1,115 +0,0 @@ -/* - 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 java.sql.{PreparedStatement, ResultSet} -import org.apache.hadoop.mapreduce.lib.db.DBWritable -import org.apache.phoenix.schema.types.{PBinary, PDataType, PDate, PVarbinary, PhoenixArray} -import org.apache.phoenix.util.ColumnInfo -import org.joda.time.DateTime -import scala.collection.{mutable, immutable} - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class PhoenixRecordWritable(columnMetaDataList: List[ColumnInfo]) extends DBWritable { - val upsertValues = mutable.ArrayBuffer[Any]() - val resultMap = mutable.Map[String, AnyRef]() - - def result : immutable.Map[String, AnyRef] = { - resultMap.toMap - } - - override def write(statement: PreparedStatement): Unit = { - // Make sure we at least line up in size - if(upsertValues.length != columnMetaDataList.length) { - throw new UnsupportedOperationException( - s"Upsert values ($upsertValues) do not match the specified columns (columnMetaDataList)" - ) - } - - // Correlate each value (v) to a column type (c) and an index (i) - upsertValues.zip(columnMetaDataList).zipWithIndex.foreach { - case ((v, c), i) => { - if (v != null) { - - // Both Java and Joda dates used to work in 4.2.3, but now they must be java.sql.Date - // Can override any other types here as needed - val (finalObj, finalType) = v match { - case dt: DateTime => (new java.sql.Date(dt.getMillis), PDate.INSTANCE) - case d: java.util.Date => (new java.sql.Date(d.getTime), PDate.INSTANCE) - case _ => (v, c.getPDataType) - } - - - // Helper method to create an SQL array for a specific PDatatype, and set it on the statement - def setArrayInStatement(obj: Array[AnyRef]): Unit = { - // Create a java.sql.Array, need to lookup the base sql type name - val sqlArray = statement.getConnection.createArrayOf( - PDataType.arrayBaseType(finalType).getSqlTypeName, - obj - ) - statement.setArray(i + 1, sqlArray) - } - - // Determine whether to save as an array or object - (finalObj, finalType) match { - case (obj: Array[AnyRef], _) => setArrayInStatement(obj) - case (obj: mutable.ArrayBuffer[AnyVal], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef]).toArray) - case (obj: mutable.ArrayBuffer[AnyRef], _) => setArrayInStatement(obj.toArray) - case (obj: mutable.WrappedArray[AnyVal], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef]).toArray) - case (obj: mutable.WrappedArray[AnyRef], _) => setArrayInStatement(obj.toArray) - case (obj: Array[Int], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Long], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Char], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Short], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Float], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case (obj: Array[Double], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - // PVarbinary and PBinary come in as Array[Byte] but they're SQL objects - case (obj: Array[Byte], _ : PVarbinary) => statement.setObject(i + 1, obj) - case (obj: Array[Byte], _ : PBinary) => statement.setObject(i + 1, obj) - // Otherwise set as array type - case (obj: Array[Byte], _) => setArrayInStatement(obj.map(_.asInstanceOf[AnyRef])) - case _ => statement.setObject(i + 1, finalObj) - } - } else { - statement.setNull(i + 1, c.getSqlType) - } - } - } - } - - override def readFields(resultSet: ResultSet): Unit = { - val metadata = resultSet.getMetaData - for(i <- 1 to metadata.getColumnCount) { - - // Return the contents of a PhoenixArray, if necessary - val value = resultSet.getObject(i) match { - case x: PhoenixArray => x.getArray - case y => y - } - - // Put a (ColumnLabel -> value) entry in the result map - resultMap(metadata.getColumnLabel(i)) = value - } - } - - def add(value: Any): Unit = { - upsertValues.append(value) - } - - // Empty constructor for MapReduce - def this() = { - this(List[ColumnInfo]()) - } - -} 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 deleted file mode 100644 index aacd4600..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.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.types.StructType -import org.apache.spark.sql.{Row, SQLContext} - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Boolean = false)(@transient val sqlContext: SQLContext) - extends BaseRelation with PrunedFilteredScan { - - /* - This is the buildScan() implementing Spark's PrunedFilteredScan. - Spark SQL queries with columns or predicates specified will be pushed down - to us here, and we can pass that on to Phoenix. According to the docs, this - is an optimization, and the filtering/pruning will be re-evaluated again, - but this prevents having to load the whole table into Spark first. - */ - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { - val(pushedFilters, _, _) = new FilterExpressionCompiler().pushFilters(filters) - new PhoenixRDD( - sqlContext.sparkContext, - tableName, - requiredColumns, - Some(pushedFilters), - Some(zkUrl), - new Configuration(), - dateAsTimestamp - ).toDataFrame(sqlContext).rdd - } - - // Required by BaseRelation, this will return the full schema for a table - override def schema: StructType = { - new PhoenixRDD( - sqlContext.sparkContext, - tableName, - Seq(), - None, - Some(zkUrl), - new Configuration(), - dateAsTimestamp - ).toDataFrame(sqlContext).schema - } - - - override def unhandledFilters(filters: Array[Filter]): Array[Filter] = { - val (_, unhandledFilters, _) = new FilterExpressionCompiler().pushFilters(filters) - unhandledFilters - } - -} diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala deleted file mode 100644 index b0735218..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.hadoop.io.NullWritable -import org.apache.phoenix.mapreduce.PhoenixOutputFormat -import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil -import org.apache.spark.rdd.RDD - -import scala.collection.JavaConversions._ - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class ProductRDDFunctions[A <: Product](data: RDD[A]) extends Serializable { - - def saveToPhoenix(tableName: String, cols: Seq[String], - conf: Configuration = new Configuration, zkUrl: Option[String] = None, tenantId: Option[String] = None) - : Unit = { - - // Create a configuration object to use for saving - @transient val outConfig = ConfigurationUtil.getOutputConfiguration(tableName, cols, zkUrl, tenantId, Some(conf)) - - // Retrieve the zookeeper URL - val zkUrlFinal = ConfigurationUtil.getZookeeperURL(outConfig) - - // Map the row objects into PhoenixRecordWritable - val phxRDD = data.mapPartitions{ rows => - - // Create a within-partition config to retrieve the ColumnInfo list - @transient val partitionConfig = ConfigurationUtil.getOutputConfiguration(tableName, cols, zkUrlFinal, tenantId) - @transient val columns = PhoenixConfigurationUtil.getUpsertColumnMetadataList(partitionConfig).toList - - rows.map { row => - val rec = new PhoenixRecordWritable(columns) - row.productIterator.foreach { e => rec.add(e) } - (null, rec) - } - } - - // Save it - phxRDD.saveAsNewAPIHadoopFile( - Option( - conf.get("mapreduce.output.fileoutputformat.outputdir") - ).getOrElse( - Option(conf.get("mapred.output.dir")).getOrElse("") - ), - classOf[NullWritable], - classOf[PhoenixRecordWritable], - classOf[PhoenixOutputFormat[PhoenixRecordWritable]], - outConfig - ) - } -} \ No newline at end of file diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala deleted file mode 100644 index 1b377abe..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class SparkContextFunctions(@transient val sc: SparkContext) extends Serializable { - - /* - This will return an RDD of Map[String, AnyRef], where the String key corresponds to the column - name and the AnyRef value will be a java.sql type as returned by Phoenix - - 'table' is the corresponding Phoenix table - 'columns' is a sequence of of columns to query - 'predicate' is a set of statements to go after a WHERE clause, e.g. "TID = 123" - 'zkUrl' is an optional Zookeeper URL to use to connect to Phoenix - 'conf' is a Hadoop Configuration object. If zkUrl is not set, the "hbase.zookeeper.quorum" - property will be used - */ - - def phoenixTableAsRDD(table: String, columns: Seq[String], predicate: Option[String] = None, - zkUrl: Option[String] = None, tenantId: Option[String] = None, conf: Configuration = new Configuration()) - : RDD[Map[String, AnyRef]] = { - - // Create a PhoenixRDD, but only return the serializable 'result' map - new PhoenixRDD(sc, table, columns, predicate, zkUrl, conf, tenantId = tenantId).map(_.result) - } -} \ No newline at end of file diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala deleted file mode 100644 index f9154adf..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - 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.hadoop.conf.Configuration -import org.apache.spark.sql.{DataFrame, SQLContext} - -@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") -class SparkSqlContextFunctions(@transient val sqlContext: SQLContext) extends Serializable { - - /* - This will return a Spark DataFrame, with Phoenix types converted Spark SQL catalyst types - - 'table' is the corresponding Phoenix table - 'columns' is a sequence of of columns to query - 'predicate' is a set of statements to go after a WHERE clause, e.g. "TID = 123" - 'zkUrl' is an optional Zookeeper URL to use to connect to Phoenix - 'conf' is a Hadoop Configuration object. If zkUrl is not set, the "hbase.zookeeper.quorum" - property will be used - */ - def phoenixTableAsDataFrame(table: String, columns: Seq[String], - predicate: Option[String] = None, - zkUrl: Option[String] = None, - tenantId: Option[String] = None, - conf: Configuration = new Configuration): DataFrame = { - - // Create the PhoenixRDD and convert it to a DataFrame - new PhoenixRDD(sqlContext.sparkContext, table, columns, predicate, zkUrl, conf, tenantId = tenantId) - .toDataFrame(sqlContext) - } -} \ No newline at end of file diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/package.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/package.scala deleted file mode 100644 index 3fed79ea..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/package.scala +++ /dev/null @@ -1,36 +0,0 @@ -/* - 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 - -import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext} - -package object spark { - implicit def toProductRDDFunctions[A <: Product](rdd: RDD[A]): ProductRDDFunctions[A] = { - new ProductRDDFunctions[A](rdd) - } - - implicit def toSparkContextFunctions(sc: SparkContext): SparkContextFunctions = { - new SparkContextFunctions(sc) - } - - implicit def toSparkSqlContextFunctions(sqlContext: SQLContext): SparkSqlContextFunctions = { - new SparkSqlContextFunctions(sqlContext) - } - - implicit def toDataFrameFunctions(data: DataFrame): DataFrameFunctions = { - new DataFrameFunctions(data) - } -} diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/FilterExpressionCompiler.scala similarity index 95% rename from phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala rename to phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/FilterExpressionCompiler.scala index 77c172ac..713b8823 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/FilterExpressionCompiler.scala @@ -15,16 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.phoenix.spark +package org.apache.phoenix.spark.sql.connector -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/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-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/SparkSchemaUtil.scala similarity index 83% rename from phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala rename to phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/SparkSchemaUtil.scala index 19535bfc..6e73a743 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/sql/connector/SparkSchemaUtil.scala @@ -15,21 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.phoenix.spark +package org.apache.phoenix.spark.sql.connector 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) {