diff --git a/phoenix5-spark/README.md b/phoenix5-spark/README.md index 0d2ebcca..97a9941a 100644 --- a/phoenix5-spark/README.md +++ b/phoenix5-spark/README.md @@ -331,10 +331,9 @@ 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. - The `"org.apache.phoenix.spark"` datasource does not accept the `"jdbcUrl"` parameter, - only `"zkUrl"` +- DataSourceV1 implementation was removed, +source type `"org.apache.phoenix.spark"` +use the DatasourceV2 since connector 6.0.0 release. - 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, diff --git a/phoenix5-spark/pom.xml b/phoenix5-spark/pom.xml index 9d617e78..1e5878f1 100644 --- a/phoenix5-spark/pom.xml +++ b/phoenix5-spark/pom.xml @@ -84,12 +84,6 @@ ${spark.version} provided - - org.apache.spark - spark-tags_${scala.binary.version} - ${spark.version} - provided - org.apache.spark spark-catalyst_${scala.binary.version} @@ -177,39 +171,6 @@ - - org.apache.hadoop - hadoop-mapreduce-client-core - provided - - - log4j - log4j - - - javax.servlet - servlet-api - - - javax.servlet.jsp - jsp-api - - - org.jruby - jruby-complete - - - org.jboss.netty - netty - - - io.netty - netty - - - - - org.apache.hbase hbase-client @@ -371,11 +332,6 @@ slf4j-api provided - - joda-time - joda-time - ${jodatime.version} - 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 28f5f621..e7e5c8e5 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 @@ -73,10 +73,8 @@ public Configuration getConfiguration(Configuration confToClone) { @Test public void basicWriteAndReadBackTest() throws SQLException { - 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 = SparkUtil.getSparkSession(); String tableName = generateUniqueName(); try (Connection conn = DriverManager.getConnection(getUrl()); @@ -85,141 +83,122 @@ public void basicWriteAndReadBackTest() throws SQLException { "CREATE TABLE " + tableName + " (id INTEGER PRIMARY KEY, v1 VARCHAR)"); } - try (SparkSession spark = sqlContext.sparkSession()) { + StructType schema = + new StructType(new StructField[] { + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("v1", DataTypes.StringType, false, Metadata.empty()) }); - StructType schema = - new StructType(new StructField[] { - new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), - new StructField("v1", DataTypes.StringType, false, Metadata.empty()) }); + // Use old zkUrl + Dataset df1 = + spark.createDataFrame( + Arrays.asList(RowFactory.create(1, "x")), + schema); - // Use old zkUrl - Dataset df1 = - spark.createDataFrame( - Arrays.asList(RowFactory.create(1, "x")), - schema); + df1.write().format("phoenix").mode(SaveMode.Overwrite) + .option("table", tableName) + .option(ZOOKEEPER_URL, getUrl()) + .save(); + + // Use jdbcUrl + // In Phoenix 5.2+ getUrl() return a JDBC URL, in earlier versions it returns a ZK + // quorum + String jdbcUrl = getUrl(); + if (!jdbcUrl.startsWith(JDBC_PROTOCOL)) { + jdbcUrl = JDBC_PROTOCOL_ZK + JDBC_PROTOCOL_SEPARATOR + jdbcUrl; + } + Dataset df2 = + spark.createDataFrame( + Arrays.asList(RowFactory.create(2, "x")), + schema); - df1.write().format("phoenix").mode(SaveMode.Overwrite) + df2.write().format("phoenix").mode(SaveMode.Overwrite) .option("table", tableName) - .option(ZOOKEEPER_URL, getUrl()) + .option(JDBC_URL, jdbcUrl) .save(); - // Use jdbcUrl - // In Phoenix 5.2+ getUrl() return a JDBC URL, in earlier versions it returns a ZK - // quorum - String jdbcUrl = getUrl(); - if (!jdbcUrl.startsWith(JDBC_PROTOCOL)) { - jdbcUrl = JDBC_PROTOCOL_ZK + JDBC_PROTOCOL_SEPARATOR + jdbcUrl; - } - Dataset df2 = - spark.createDataFrame( - Arrays.asList(RowFactory.create(2, "x")), - schema); + // Use default from hbase-site.xml + Dataset df3 = + spark.createDataFrame( + Arrays.asList(RowFactory.create(3, "x")), + schema); - df2.write().format("phoenix").mode(SaveMode.Overwrite) - .option("table", tableName) - .option(JDBC_URL, jdbcUrl) - .save(); + df3.write().format("phoenix").mode(SaveMode.Overwrite) + .option("table", tableName) + .save(); - // Use default from hbase-site.xml - Dataset df3 = - spark.createDataFrame( - Arrays.asList(RowFactory.create(3, "x")), - schema); + try (Connection conn = DriverManager.getConnection(getUrl()); + Statement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName); + assertTrue(rs.next()); + assertEquals(1, rs.getInt(1)); + assertEquals("x", rs.getString(2)); + assertTrue(rs.next()); + assertEquals(2, rs.getInt(1)); + assertEquals("x", rs.getString(2)); + assertTrue(rs.next()); + assertEquals(3, rs.getInt(1)); + assertEquals("x", rs.getString(2)); + assertFalse(rs.next()); + } - df3.write().format("phoenix").mode(SaveMode.Overwrite) + Dataset df1Read = spark.read().format("phoenix") .option("table", tableName) - .save(); - - try (Connection conn = DriverManager.getConnection(getUrl()); - Statement stmt = conn.createStatement()) { - ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName); - assertTrue(rs.next()); - assertEquals(1, rs.getInt(1)); - assertEquals("x", rs.getString(2)); - assertTrue(rs.next()); - assertEquals(2, rs.getInt(1)); - assertEquals("x", rs.getString(2)); - assertTrue(rs.next()); - assertEquals(3, rs.getInt(1)); - assertEquals("x", rs.getString(2)); - assertFalse(rs.next()); - } - - Dataset df1Read = spark.read().format("phoenix") - .option("table", tableName) - .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); - assertEquals(3l, df1Read.count()); + assertEquals(3l, df1Read.count()); - // Use jdbcUrl - Dataset df2Read = spark.read().format("phoenix") - .option("table", tableName) - .option(PhoenixDataSource.JDBC_URL, jdbcUrl) - .load(); - - assertEquals(3l, df2Read.count()); + // Use jdbcUrl + Dataset df2Read = spark.read().format("phoenix") + .option("table", tableName) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .load(); - // Use default - Dataset df3Read = spark.read().format("phoenix") - .option("table", tableName) - .load(); + assertEquals(3l, df2Read.count()); - assertEquals(3l, df3Read.count()); + // Use default + Dataset df3Read = spark.read().format("phoenix") + .option("table", tableName) + .load(); - } finally { - jsc.stop(); - } + assertEquals(3l, df3Read.count()); } @Test public void lowerCaseWriteTest() throws SQLException { - 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 = SparkUtil.getSparkSession(); String tableName = generateUniqueName(); try (Connection conn = DriverManager.getConnection(getUrl()); Statement stmt = conn.createStatement()){ stmt.executeUpdate("CREATE TABLE " + tableName + " (id INTEGER PRIMARY KEY, v1 VARCHAR, \"v1\" VARCHAR)"); } + StructType schema = new StructType(new StructField[]{ + new StructField("ID", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("V1", DataTypes.StringType, false, Metadata.empty()), + new StructField("\"v1\"", DataTypes.StringType, false, Metadata.empty()) + }); - try(SparkSession spark = sqlContext.sparkSession()) { - //Doesn't help - spark.conf().set("spark.sql.caseSensitive", true); - - StructType schema = new StructType(new StructField[]{ - new StructField("ID", DataTypes.IntegerType, false, Metadata.empty()), - new StructField("V1", DataTypes.StringType, false, Metadata.empty()), - new StructField("\"v1\"", DataTypes.StringType, false, Metadata.empty()) - }); - - Dataset df = spark.createDataFrame( - Arrays.asList( - RowFactory.create(1, "x", "y")), - schema); - - df.write() - .format("phoenix") - .mode(SaveMode.Overwrite) - .option("table", tableName) - .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER,"true") - .option(JDBC_URL, getUrl()) - .save(); - - try (Connection conn = DriverManager.getConnection(getUrl()); - Statement stmt = conn.createStatement()) { - ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName); - assertTrue(rs.next()); - assertEquals(1, rs.getInt(1)); - assertEquals("x", rs.getString(2)); - assertEquals("y", rs.getString(3)); - assertFalse(rs.next()); - } + Dataset df = spark.createDataFrame( + Arrays.asList( + RowFactory.create(1, "x", "y")), + schema); + df.write() + .format("phoenix") + .mode(SaveMode.Overwrite) + .option("table", tableName) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER,"true") + .option(JDBC_URL, getUrl()) + .save(); - } finally { - jsc.stop(); + try (Connection conn = DriverManager.getConnection(getUrl()); + Statement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName); + assertTrue(rs.next()); + assertEquals(1, rs.getInt(1)); + assertEquals("x", rs.getString(2)); + assertEquals("y", rs.getString(3)); + assertFalse(rs.next()); } } 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 776cc77f..760bf559 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 @@ -67,7 +67,7 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter lazy val jdbcUrl = PhoenixSparkITHelper.getUrl - lazy val quorumAddress = ConfigurationUtil.getZookeeperURL(hbaseConfiguration).get + lazy val quorumAddress = PhoenixSparkITHelper.getUrl // Runs SQL commands located in the file defined in the sqlSource argument // Optional argument tenantId used for running tenant-specific SQL diff --git a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala deleted file mode 100644 index 0450ce05..00000000 --- a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala +++ /dev/null @@ -1,743 +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.mapreduce.util.PhoenixConfigurationUtil -import org.apache.phoenix.schema.types.{PSmallintArray, PVarchar} -import org.apache.phoenix.spark.datasource.v2.reader.PhoenixTestingInputPartitionReader -import org.apache.phoenix.spark.datasource.v2.writer.PhoenixTestingDataSourceWriter -import org.apache.phoenix.spark.datasource.v2.{PhoenixDataSource, PhoenixTestingDataSource} -import org.apache.phoenix.util.{ColumnInfo, SchemaUtil} -import org.apache.spark.SparkException -import org.apache.spark.sql.types._ -import org.apache.spark.sql.{Row, SaveMode} - -import java.sql.DriverManager -import java.time.{LocalDate, ZoneId} -import java.util.Date -import scala.collection.mutable -import scala.collection.mutable.ListBuffer - -/** - * Note: If running directly from an IDE, these are the recommended VM parameters: - * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m - */ -class PhoenixSparkDatasourceV1IT extends AbstractPhoenixSparkIT { - - test("Can persist data with case sensitive columns (like in avro schema)") { - val df = spark.createDataFrame( - Seq( - (1, 1, "test_child_1"), - (2, 1, "test_child_2"))). - // column names are case sensitive - toDF("ID", "TABLE3_ID", "t2col1") - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "TABLE3", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, - PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true")) - .mode(SaveMode.Overwrite) - .save() - - - // Verify results - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT * FROM TABLE3") - - val checkResults = List((1, 1, "test_child_1"), (2, 1, "test_child_2")) - val results = ListBuffer[(Long, Long, String)]() - while (rs.next()) { - results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) - } - stmt.close() - - results.toList shouldEqual checkResults - } - - test("Can convert Phoenix schema") { - val phoenixSchema = List( - new ColumnInfo("varcharColumn", PVarchar.INSTANCE.getSqlType) - ) - - val catalystSchema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(phoenixSchema) - - val expected = new StructType(List(StructField("varcharColumn", StringType, nullable = true)).toArray) - - catalystSchema shouldEqual expected - } - - test("Can convert arrays of Short type in Phoenix schema") { - val phoenixSchema = List( - new ColumnInfo("arrayshortColumn", PSmallintArray.INSTANCE.getSqlType) - ) - - val catalystSchema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(phoenixSchema) - - val expected = new StructType(List(StructField("arrayshortColumn", ArrayType(ShortType, true), nullable = true)).toArray) - - catalystSchema shouldEqual expected - } - - test("Can create schema RDD and execute query") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df1.createOrReplaceTempView("sql_table_1") - - val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df2.createOrReplaceTempView("sql_table_2") - - val sqlRdd = spark.sql( - """ - |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 - |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin - ) - - val count = sqlRdd.count() - - count shouldEqual 6L - } - - ignore("Ordering by pk columns should not require sorting") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - df1.createOrReplaceTempView("TABLE1") - - val sqlRdd = spark.sql("SELECT * FROM TABLE1 ORDER BY ID, COL1") - val plan = sqlRdd.queryExecution.sparkPlan - // verify the spark plan doesn't have a sort - assert(!plan.toString.contains("Sort")) - - val expectedResults = Array(Row.fromSeq(Seq(1, "test_row_1")), Row.fromSeq(Seq(2, "test_row_2"))) - val actual = sqlRdd.collect() - - actual shouldEqual expectedResults - } - - test("Verify correct number of partitions are created") { - val conn = DriverManager.getConnection(PhoenixSparkITHelper.getUrl) - val ddl = "CREATE TABLE SPLIT_TABLE (id VARCHAR NOT NULL PRIMARY KEY, val VARCHAR) split on ('e','j','o')" - conn.createStatement.execute(ddl) - val keys = Array("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s", - "t", "u", "v", "w", "x", "y", "z") - for (key <- keys) { - conn.createStatement.execute("UPSERT INTO SPLIT_TABLE VALUES('" + key + "', '" + key + "')") - } - conn.commit() - - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "SPLIT_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - df1.createOrReplaceTempView("SPLIT_TABLE") - val sqlRdd = spark.sql("SELECT * FROM SPLIT_TABLE") - val numPartitions = sqlRdd.rdd.partitions.size - - numPartitions shouldEqual 4 - } - - test("Can create schema RDD and execute query on case sensitive table (no config)") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> SchemaUtil.getEscapedArgument("table4"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df1.createOrReplaceTempView("table4") - - val sqlRdd = spark.sql("SELECT id FROM table4") - - val count = sqlRdd.count() - - count shouldEqual 2L - } - - test("Can create schema RDD and execute constrained query") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df1.createOrReplaceTempView("sql_table_1") - - val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = 1") - - df2.createOrReplaceTempView("sql_table_2") - - val sqlRdd = spark.sql( - """ - |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 - |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin - ) - - val count = sqlRdd.count() - - count shouldEqual 1L - } - - test("Can create schema RDD with predicate that will never match") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = -1") - - df1.createOrReplaceTempView("table3") - - val sqlRdd = spark.sql("SELECT * FROM table3") - - val count = sqlRdd.count() - - count shouldEqual 0L - } - - test("Can create schema RDD with complex predicate") { - val predicate = "ID > 0 AND TIMESERIES_KEY BETWEEN " + - "CAST(TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP) AND " + - "CAST(TO_DATE('1990-01-30 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP)" - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map("table" -> "DATE_PREDICATE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - .filter(predicate) - - df1.createOrReplaceTempView("date_predicate_test_table") - - val sqlRdd = spark.sqlContext.sql("SELECT * FROM date_predicate_test_table") - - val count = sqlRdd.count() - - count shouldEqual 0L - } - - test("Can query an array table") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df1.createOrReplaceTempView("ARRAY_TEST_TABLE") - - val sqlRdd = spark.sql("SELECT * FROM ARRAY_TEST_TABLE") - - val count = sqlRdd.count() - - // get row 0, column 1, which should be "VCARRAY" - val arrayValues = sqlRdd.collect().apply(0).apply(1) - - arrayValues should equal(Array("String1", "String2", "String3")) - - count shouldEqual 1L - } - - test("Can read a table as an RDD") { - val rdd1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - val count = rdd1.count() - - val arrayValues = rdd1.take(1)(0)(1) - - arrayValues should equal(Array("String1", "String2", "String3")) - - count shouldEqual 1L - } - - test("Can save to phoenix table from Spark without specifying all the columns") { - val dataSet = List(Row(1L, "1", 1), Row(2L, "2", 2), Row(3L, "3", 3)) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("COL1", StringType), - StructField("COL2", IntegerType))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT ID, COL1, COL2 FROM OUTPUT_TEST_TABLE") - val results = ListBuffer[Row]() - while (rs.next()) { - results.append(Row(rs.getLong(1), rs.getString(2), rs.getInt(3))) - } - - // Verify they match - (0 to results.size - 1).foreach { i => - dataSet(i) shouldEqual results(i) - } - } - - test("Can save dates to Phoenix using java.sql.Date") { - val date = java.sql.Date.valueOf("2016-09-30") - - // Since we are creating a Row we have to use java.sql.date - // java.util.date or joda.DateTime is not supported - val dataSet = Seq(Row(1L, "1", 1, date), Row(2L, "2", 2, date)) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("COL1", StringType), - StructField("COL2", IntegerType), - StructField("COL3", DateType))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT COL3 FROM OUTPUT_TEST_TABLE WHERE ID = 1 OR ID = 2 ORDER BY ID ASC") - val results = ListBuffer[java.sql.Date]() - while (rs.next()) { - results.append(rs.getDate(1)) - } - - // Verify the epochs are equal - results(0).getTime shouldEqual date.getTime - results(1).getTime shouldEqual date.getTime - } - - test("Can infer schema without defining columns") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - df.schema("ID").dataType shouldEqual LongType - df.schema("TABLE1_ID").dataType shouldEqual LongType - df.schema("t2col1").dataType shouldEqual StringType - } - - test("Spark SQL can use Phoenix as a data source with no schema specified") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - df.count() shouldEqual 2 - df.schema("ID").dataType shouldEqual LongType - df.schema("COL1").dataType shouldEqual StringType - } - - - test("Can persist a dataframe") { - // Load from TABLE1 - val df = spark.sqlContext.read.format("org.apache.phoenix.spark").options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - // Save to TABLE1_COPY - df - .write - .format("org.apache.phoenix.spark") - .mode(SaveMode.Overwrite) - .option("table", "TABLE1_COPY") - .option(PhoenixDataSource.ZOOKEEPER_URL, quorumAddress) - .save() - - // Verify results - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT * FROM TABLE1_COPY") - - val checkResults = List((1L, "test_row_1"), (2, "test_row_2")) - val results = ListBuffer[(Long, String)]() - while (rs.next()) { - results.append((rs.getLong(1), rs.getString(2))) - } - stmt.close() - - results.toList shouldEqual checkResults - } - - test("Can save arrays back to phoenix") { - val dataSet = List(Row(2L, Array("String1", "String2", "String3"))) - val schema = StructType(Seq( - StructField("ID", LongType, nullable = false), - StructField("VCARRAY", ArrayType(StringType, true)) - )) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT VCARRAY FROM ARRAY_TEST_TABLE WHERE ID = 2") - rs.next() - val sqlArray = rs.getArray(1).getArray().asInstanceOf[Array[String]] - - // Verify the arrays are equal - sqlArray shouldEqual dataSet(0).get(1) - } - - test("Can read from table with schema and escaped table name") { - // Manually escape - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map("table" -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - - var count = df1.count() - - count shouldEqual 1L - - // Use SchemaUtil - val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( - Map("table" -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load() - - count = df2.count() - - count shouldEqual 1L - } - - test("Ensure DataFrame field normalization (PHOENIX-2196)") { - val rdd1 = spark.sparkContext - .parallelize(Seq((1L, 1L, "One"), (2L, 2L, "Two"))) - .map(p => Row(p._1, p._2, p._3)) - - val schema = StructType(Seq( - StructField("id", LongType, nullable = false), - StructField("table1_id", LongType, nullable = true), - StructField("\"t2col1\"", StringType, nullable = true) - )) - - val df = spark.sqlContext.createDataFrame(rdd1, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - } - - test("Ensure Dataframe supports LIKE and IN filters (PHOENIX-2328)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark").options(Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - // Prefix match - val res1 = df.filter("COL1 like 'test_row_%'") - val plan = res1.groupBy().count().queryExecution.sparkPlan - res1.count() shouldEqual 2 - - // Suffix match - val res2 = df.filter("COL1 like '%_1'") - res2.count() shouldEqual 1 - res2.first.getString(1) shouldEqual "test_row_1" - - // Infix match - val res3 = df.filter("COL1 like '%_row_%'") - res3.count() shouldEqual 2 - - // Not like, match none - val res4 = df.filter("COL1 not like '%_row_%'") - res4.count() shouldEqual 0 - - // Not like, match all - val res5 = df.filter("COL1 not like '%_wor_%'") - res5.count() shouldEqual 2 - - // "IN", match all - val res6 = df.filter("COL1 in ('test_row_1', 'test_row_2')") - res6.count() shouldEqual 2 - - // "IN", match none - val res7 = df.filter("COL1 in ('foo', 'bar')") - res7.count() shouldEqual 0 - - // AND (and not again) - val res8 = df.filter("COL1 like '%_row_%' AND COL1 not like '%_1'") - res8.count() shouldEqual 1 - res8.first.getString(1) shouldEqual "test_row_2" - - // OR - val res9 = df.filter("COL1 like '%_1' OR COL1 like '%_2'") - res9.count() shouldEqual 2 - } - - test("Can load decimal types with accurate precision and scale (PHOENIX-2288)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map("table" -> "TEST_DECIMAL", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - assert(df.select("COL1").first().getDecimal(0) == BigDecimal("123.456789").bigDecimal) - } - - test("Can load small and tiny integer types (PHOENIX-2426)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map("table" -> "TEST_SMALL_TINY", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - assert(df.select("COL1").first().getShort(0).toInt == 32767) - assert(df.select("COL2").first().getByte(0).toInt == 127) - } - - test("Can save arrays from custom dataframes back to phoenix") { - val dataSet = List(Row(2L, Array("String1", "String2", "String3"), Array(1, 2, 3))) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("VCARRAY", ArrayType(StringType)), - StructField("INTARRAY", ArrayType(IntegerType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "ARRAYBUFFER_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT VCARRAY, INTARRAY FROM ARRAYBUFFER_TEST_TABLE WHERE ID = 2") - rs.next() - val stringArray = rs.getArray(1).getArray().asInstanceOf[Array[String]] - val intArray = rs.getArray(2).getArray().asInstanceOf[Array[Int]] - - // Verify the arrays are equal - stringArray shouldEqual dataSet(0).getAs[Array[String]](1) - intArray shouldEqual dataSet(0).getAs[Array[Int]](2) - } - - test("Can save arrays of AnyVal type back to phoenix") { - val dataSet = List(Row(2L, Array(1, 2, 3), Array(1L, 2L, 3L))) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("INTARRAY", ArrayType(IntegerType)), - StructField("BIGINTARRAY", ArrayType(LongType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "ARRAY_ANYVAL_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT INTARRAY, BIGINTARRAY FROM ARRAY_ANYVAL_TEST_TABLE WHERE ID = 2") - rs.next() - val intArray = rs.getArray(1).getArray().asInstanceOf[Array[Int]] - val longArray = rs.getArray(2).getArray().asInstanceOf[Array[Long]] - - // Verify the arrays are equal - intArray shouldEqual dataSet(0).get(1) - longArray shouldEqual dataSet(0).get(2) - } - - test("Can save arrays of Byte type back to phoenix") { - val dataSet = List(Row(2L, Array(1.toByte, 2.toByte, 3.toByte))) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("BYTEARRAY", ArrayType(ByteType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "ARRAY_BYTE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT BYTEARRAY FROM ARRAY_BYTE_TEST_TABLE WHERE ID = 2") - rs.next() - val byteArray = rs.getArray(1).getArray().asInstanceOf[Array[Byte]] - - // Verify the arrays are equal - byteArray shouldEqual dataSet(0).get(1) - } - - test("Can save arrays of Short type back to phoenix") { - val dataSet = List(Row(2L, Array(1.toShort, 2.toShort, 3.toShort))) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("SHORTARRAY", ArrayType(ShortType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "ARRAY_SHORT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT SHORTARRAY FROM ARRAY_SHORT_TEST_TABLE WHERE ID = 2") - rs.next() - val shortArray = rs.getArray(1).getArray().asInstanceOf[Array[Short]] - - // Verify the arrays are equal - shortArray shouldEqual dataSet(0).get(1) - } - - test("Can save binary types back to phoenix") { - val dataSet = List(Row(2L, Array[Byte](1), Array[Byte](1, 2, 3), Array[Array[Byte]](Array[Byte](1), Array[Byte](2)))) - - val schema = StructType( - Seq(StructField("ID", LongType, false), - StructField("BIN", BinaryType), - StructField("VARBIN", BinaryType), - StructField("BINARRAY", ArrayType(BinaryType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "VARBINARY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT BIN, VARBIN, BINARRAY FROM VARBINARY_TEST_TABLE WHERE ID = 2") - rs.next() - val byte = rs.getBytes("BIN") - val varByte = rs.getBytes("VARBIN") - val varByteArray = rs.getArray("BINARRAY").getArray().asInstanceOf[Array[Array[Byte]]] - - // Verify the arrays are equal - byte shouldEqual dataSet(0).get(1) - varByte shouldEqual dataSet(0).get(2) - varByteArray shouldEqual dataSet(0).get(3) - } - - test("Can load and filter Phoenix DATE columns through DataFrame API") { - val df = spark.sqlContext.read - .format("org.apache.phoenix.spark") - .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - val dt = df.select("COL1").first().getDate(0).getTime - val expected = LocalDate.of(2021,1,1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - - // NOTE: Spark DateType drops hour, minute, second, as per the java.sql.Date spec - // Use 'dateAsTimestamp' option to coerce DATE to TIMESTAMP without losing resolution - - // Note that Spark also applies the timezone offset to the returned date epoch. Rather than perform timezone - // gymnastics, just make sure we're within 24H of the epoch generated just now - assert(expected == dt) - - df.createOrReplaceTempView("DATE_TEST") - val df2 = spark.sql("SELECT * FROM DATE_TEST WHERE COL1 > TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss')") - assert(df2.count() == 1L) - } - - test("Filter operation doesn't work for column names containing a white space (PHOENIX-2547)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map("table" -> SchemaUtil.getEscapedArgument("space"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - val res = df.filter(df.col("first name").equalTo("xyz")) - // Make sure we got the right value back - assert(res.collectAsList().size() == 1L) - } - - test("Spark Phoenix cannot recognize Phoenix view fields (PHOENIX-2290)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - df.createOrReplaceTempView("temp") - - // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions - // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle - - val res1 = spark.sql("select * from temp where salary = '10000' ") - assert(res1.collectAsList().size() == 1L) - - val res2 = spark.sql("select * from temp where \"salary\" = '10000' ") - assert(res2.collectAsList().size() == 0L) - - val res3 = spark.sql("select * from temp where salary > '10000' ") - assert(res3.collectAsList().size() == 2L) - } - - test("Queries with small case column-names return empty result-set when working with Spark Datasource Plugin (PHOENIX-2336)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - - // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions - // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle - - val res1 = df.filter(df.col("first name").equalTo("foo")) - assert(res1.collectAsList().size() == 1L) - - val res2 = df.filter("\"first name\" = 'foo'") - assert(res2.collectAsList().size() == 0L) - - val res3 = df.filter("salary = '10000'") - assert(res3.collectAsList().size() == 1L) - - val res4 = df.filter("salary > '10000'") - assert(res4.collectAsList().size() == 2L) - } - - test("Can coerce Phoenix DATE columns to TIMESTAMP through DataFrame API") { - val df = spark.sqlContext.read - .format("org.apache.phoenix.spark") - .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, "dateAsTimestamp" -> "true")) - .load - val dtRes = df.select("COL1").first() - val ts = dtRes.getTimestamp(0).getTime - val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - - assert(expected == ts) - } - - test("Can load Phoenix Time columns through DataFrame API") { - val df = spark.sqlContext.read - .format("org.apache.phoenix.spark") - .options(Map("table" -> "TIME_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - val time = df.select("COL1").first().getTimestamp(0).getTime - val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - - assert(expected == time) - } - - test("can read all Phoenix data types") { - val df = spark.sqlContext.read - .format("org.apache.phoenix.spark") - .options(Map("table" -> "GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - - df.write - .format("org.apache.phoenix.spark") - .options(Map("table" -> "OUTPUT_GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - df.count() shouldEqual 1 - } - -} diff --git a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1IT.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1IT.scala new file mode 100644 index 00000000..68f092b1 --- /dev/null +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1IT.scala @@ -0,0 +1,148 @@ +/* + 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.rdd.RDD +import org.apache.spark.sql.SaveMode + +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 PhoenixSparkV1IT extends AbstractPhoenixSparkIT { + + test("Can persist data with case sensitive columns (like in avro schema)") { + val df = spark.createDataFrame( + Seq( + (1, 1, "test_child_1"), + (2, 1, "test_child_2"))). + // column names are case sensitive + toDF("ID", "TABLE3_ID", "t2col1") + df. + write + .format("org.apache.phoenix.spark") + .option(PhoenixDataSource.TABLE,"TABLE3") + .option(PhoenixDataSource.JDBC_URL,jdbcUrl) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER,"true") + .mode(SaveMode.Overwrite) + .save() + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3") + + val checkResults = List((1, 1, "test_child_1"), (2, 1, "test_child_2")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + + test("Can persist data with case sensitive columns using utility method") { + val df = spark.createDataFrame( + Seq( + (1, 1, "test_child_1"), + (2, 1, "test_child_2"))). + // column names are case sensitive + toDF("ID", "TABLE3_ID", "t2col1") + df + .saveToPhoenix(tableName = "TABLE3", + zkUrl = Some(jdbcUrl), + conf = hbaseConfiguration, + skipNormalizingIdentifier = true) + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3") + + val checkResults = List((1, 1, "test_child_1"), (2, 1, "test_child_2")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + + test("Can create schema RDD and execute query") { + val df1 = spark.read.format("org.apache.phoenix.spark") + .options(Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("sql_table_1") + + val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df2.createOrReplaceTempView("sql_table_2") + + val sqlRdd = spark.sql( + """ + |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 + |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin + ) + + val count = sqlRdd.count() + + count shouldEqual 6L + } + + test("can read selected columns As RDD with filter") { + val expected: Seq[Map[String, AnyRef]] = Seq( + Map( + "ID" -> 1.asInstanceOf[AnyRef], + "T2COL1" -> "test_child_1".asInstanceOf[AnyRef] + ) + ) + val rdd: RDD[Map[String, AnyRef]] = spark.sparkContext.phoenixTableAsRDD( + table = "TABLE2", + columns = Seq("ID", "T2COL1"), + zkUrl = Some(quorumAddress), + conf = hbaseConfiguration, + predicate = Some("ID = 1")) + + rdd.collect().toSeq shouldEqual expected + } + + test("Can persist a dataframe") { + // Load from TABLE1 + val df = spark.sqlContext.phoenixTableAsDataFrame( + "TABLE1", + Nil, + zkUrl = Some(quorumAddress), + conf = hbaseConfiguration) + // Save to TABLE1_COPY + df + .saveToPhoenix(tableName = "TABLE1_COPY", zkUrl = Some(quorumAddress), conf = hbaseConfiguration) + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE1_COPY") + + val checkResults = List((1L, "test_row_1"), (2, "test_row_2")) + val results = ListBuffer[(Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getString(2))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + +} diff --git a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1ITTenantSpecific.scala similarity index 97% rename from phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala rename to phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1ITTenantSpecific.scala index 28907dbc..1daee5f2 100644 --- a/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala +++ b/phoenix5-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1ITTenantSpecific.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ListBuffer * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m * */ -class PhoenixSparkDatasourceV1ITTenantSpecific extends AbstractPhoenixSparkIT { +class PhoenixSparkV1ITTenantSpecific extends AbstractPhoenixSparkIT { // Tenant-specific schema info val OrgIdCol = "ORGANIZATION_ID" 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 fc98bea3..8303abce 100644 --- a/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/PhoenixDataSource.java +++ b/phoenix5-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/PhoenixDataSource.java @@ -20,8 +20,6 @@ import java.util.Optional; import java.util.Properties; -import org.apache.phoenix.query.HBaseFactoryProvider; -import org.apache.phoenix.spark.ConfigurationUtil; import org.apache.phoenix.spark.datasource.v2.reader.PhoenixDataSourceReader; import org.apache.phoenix.spark.datasource.v2.writer.PhoenixDataSourceWriter; import org.apache.phoenix.util.PhoenixRuntime; 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 3638b278..dc8b9e4e 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 @@ -218,10 +218,10 @@ private List generateColumnInfo(Connection conn, String tableName) t PTable table = PhoenixRuntime.getTable(conn, SchemaUtil.normalizeFullTableName(tableName)); int startOffset = 0; - if(table.getTenantId()!=null) { + if (table.getTenantId() != null) { startOffset++; } - if(table.getBucketNum()!=null){ + if (table.getBucketNum() != null) { startOffset++; } 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 index 3b0289d2..53f2f8a4 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala @@ -14,66 +14,37 @@ 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._ +import org.apache.phoenix.spark.PhoenixDataFrameHelper.phoenixConfig +import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource +import org.apache.spark.sql.{DataFrame, SaveMode} @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 + saveToPhoenix( + tableName = parameters("table"), + zkUrl = parameters.get("zkUrl"), + tenantId = parameters.get("TenantId"), + skipNormalizingIdentifier = parameters.contains("skipNormalizingIdentifier") ) } - def getFieldArray(skipNormalizingIdentifier: Boolean = false, data: DataFrame) = { - if (skipNormalizingIdentifier) { - data.schema.fieldNames.map(x => x) - } else { - data.schema.fieldNames.map(x => SchemaUtil.normalizeIdentifier(x)) - } + def saveToPhoenix(tableName: String, + conf: Configuration = new Configuration, + zkUrl: Option[String] = None, + tenantId: Option[String] = None, + skipNormalizingIdentifier: Boolean = false): Unit = { + data + .write + .format("phoenix") + .mode(SaveMode.Overwrite) + .option(PhoenixDataSource.TABLE, tableName) + .option(PhoenixDataSource.JDBC_URL, zkUrl.orNull) + .option(PhoenixDataSource.TENANT_ID, tenantId.orNull) + .option(PhoenixDataSource.PHOENIX_CONFIGS, phoenixConfig(conf)) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER, skipNormalizingIdentifier) + .save() + } + } 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 index ccdf5956..2dd70e22 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala @@ -17,44 +17,6 @@ */ package org.apache.phoenix.spark -import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider} -import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} +import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource -@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") - } - -} +class DefaultSource extends PhoenixDataSource diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixDataFrameHelper.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixDataFrameHelper.scala new file mode 100644 index 00000000..f77fea8c --- /dev/null +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixDataFrameHelper.scala @@ -0,0 +1,77 @@ +/* + * 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.phoenix.spark.datasource.v2.PhoenixDataSource +import org.apache.spark.sql.{DataFrame, SparkSession} + +import scala.collection.JavaConverters.asScalaIteratorConverter +import scala.collection.mutable + +private[spark] object PhoenixDataFrameHelper { + + def createDataFrame(table: String, + zkUrl: Option[String] = None, + tenantId: Option[String] = None, + conf: Configuration)(implicit sparkSession: SparkSession): DataFrame = { + val sparkOptions = mutable.Map(PhoenixDataSource.TABLE -> table, + PhoenixDataSource.PHOENIX_CONFIGS -> phoenixConfig(conf)) + + if (zkUrl.isDefined) { + sparkOptions += (PhoenixDataSource.JDBC_URL -> zkUrl.get) + } + if (tenantId.isDefined) { + sparkOptions += (PhoenixDataSource.TENANT_ID -> tenantId.get) + } + + sparkSession + .read + .format("phoenix") + .options(sparkOptions) + .load() + } + + def phoenixConfig(conf: Configuration): String = { + conf + .iterator() + .asScala + .toSeq + .filter(c => (c.getValue != null && c.getValue.trim.nonEmpty) && !c.getValue.contains(",")) + .map(c => s"${c.getKey}=${c.getValue}") + .mkString(",") + } + + def withSelectExpr(columns: Seq[String], df: DataFrame): DataFrame = { + if (columns.nonEmpty) { + df.selectExpr(columns: _*) + } else { + df + } + } + + def withWhereCondition(predicate: Option[String], df: DataFrame): DataFrame = { + if (predicate.isDefined) { + df.where(predicate.get) + } else { + df + } + } + +} 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 c0af5c4a..00000000 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala +++ /dev/null @@ -1,148 +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 2edf5463..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, Filter, PrunedFilteredScan} -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 index b0735218..f12b6a34 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala @@ -14,51 +14,58 @@ 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.spark.datasource.v2.PhoenixDataSource +import org.apache.phoenix.util.PhoenixRuntime import org.apache.spark.rdd.RDD +import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, SparkSession} -import scala.collection.JavaConversions._ +import java.sql.{DriverManager, SQLException} +import java.util.Properties +import scala.collection.JavaConverters +import scala.collection.JavaConverters.{asScalaIteratorConverter, mapAsJavaMapConverter, seqAsJavaListConverter} +import scala.util.{Failure, Success, Try} @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) - } + def saveToPhoenix(tableName: String, + cols: Seq[String], + conf: Configuration = new Configuration, + zkUrl: Option[String] = None, + tenantId: Option[String] = None): Unit = { + val sparkSession: SparkSession = SparkSession.builder().config(data.sparkContext.getConf).getOrCreate() + val dsOptions = new DataSourceOptions(Map(PhoenixDataSource.JDBC_URL -> zkUrl.orNull).asJava) + val jdbcUrl = PhoenixDataSource.getJdbcUrlFromOptions(dsOptions) + val confAsMap = conf.iterator().asScala.map(c => (c.getKey -> c.getValue)).toMap.asJava + val confToSet = new Properties() + confToSet.putAll(confAsMap) + if (tenantId.isDefined) { + confToSet.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId.get) } + val schema: StructType = catalystSchema(tableName, cols, jdbcUrl, confToSet) + val dataFrame = sparkSession.createDataFrame(data.map(Row.fromTuple), schema).selectExpr(cols: _*) + new DataFrameFunctions(dataFrame) + .saveToPhoenix( + tableName = tableName, + conf = conf, + zkUrl = zkUrl, + tenantId = tenantId + ) + } - // 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 - ) + private def catalystSchema(tableName: String, columnList: Seq[String], jdbcUrl: String, overriddenProps: Properties): StructType = { + Try(DriverManager.getConnection(jdbcUrl, overriddenProps)) match { + case Success(conn) => try { + val columnInfos = PhoenixRuntime.generateColumnInfo(conn, tableName, columnList.asJava) + val columnInfoSeq = JavaConverters.asScalaIteratorConverter(columnInfos.iterator).asScala.toSeq + SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoSeq) + } catch { + case e: SQLException => + throw new RuntimeException(e) + } finally if (conn != null) conn.close() + case Failure(e) => throw new RuntimeException(e) + } } + } \ 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 index 1b377abe..75b750b5 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala @@ -16,6 +16,7 @@ package org.apache.phoenix.spark import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession @deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") class SparkContextFunctions(@transient val sc: SparkContext) extends Serializable { @@ -32,11 +33,25 @@ class SparkContextFunctions(@transient val sc: SparkContext) extends Serializabl 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) + 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]] = { + + val sparkSession = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataFrame = sparkSession.sqlContext.phoenixTableAsDataFrame( + table = table, + columns = columns, + predicate = predicate, + zkUrl = zkUrl, + tenantId = tenantId, + conf = conf + ) + + dataFrame + .rdd + .map(row => row.getValuesMap(row.schema.fieldNames)) } } \ No newline at end of file diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala index dd5dbefd..8f866136 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala @@ -33,7 +33,7 @@ object SparkSchemaUtil { new StructType(structFields.toArray) } - private def normalizeColumnName(columnName: String, doNotMapColumnFamily: Boolean ) = { + private def normalizeColumnName(columnName: String, doNotMapColumnFamily: Boolean) = { val unescapedColumnName = SchemaUtil.getUnEscapedFullColumnName(columnName) var normalizedColumnName = "" if (unescapedColumnName.indexOf(QueryConstants.NAME_SEPARATOR) < 0) { diff --git a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala index f9154adf..6899bef0 100644 --- a/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala +++ b/phoenix5-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala @@ -18,25 +18,25 @@ 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 + 'columns' is a sequence 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) + predicate: Option[String] = None, + zkUrl: Option[String] = None, + tenantId: Option[String] = None, + conf: Configuration = new Configuration): DataFrame = { + implicit val sparkSession = sqlContext.sparkSession + val df = PhoenixDataFrameHelper.createDataFrame(table, zkUrl, tenantId, conf) + val dfWithSelectColumns = PhoenixDataFrameHelper.withSelectExpr(columns, df) + PhoenixDataFrameHelper.withWhereCondition(predicate, dfWithSelectColumns) } + } \ No newline at end of file diff --git a/phoenix5-spark3/README.md b/phoenix5-spark3/README.md index a3f03271..5974da8b 100644 --- a/phoenix5-spark3/README.md +++ b/phoenix5-spark3/README.md @@ -165,7 +165,9 @@ The `save` is method on DataFrame allows passing in a data source type. You can specify which table to persist the DataFrame to. The column names are derived from the DataFrame's schema field names, and must match the Phoenix column names. -The `save` method also takes a `SaveMode` option, for which only `SaveMode.Append` is supported. +The `save` method also takes a `SaveMode` option, it is recommended to use `SaveMode.Append`. +For maintaining compatibility with source type `"org.apache.phoenix.spark"`, +`SaveMode.Overwrite` is accepted, but it behaves same way as `SaveMode.Append`. Given two Phoenix tables with the following DDL: @@ -341,10 +343,9 @@ 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. - The `"org.apache.phoenix.spark"` datasource does not accept the `"jdbcUrl"` parameter, - only `"zkUrl"` +- DataSourceV1 implementation was removed, +source type `"org.apache.phoenix.spark"` +use the DatasourceV2 since connector 6.0.0 release. - 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, diff --git a/phoenix5-spark3/pom.xml b/phoenix5-spark3/pom.xml index 8801b00f..634cb9de 100644 --- a/phoenix5-spark3/pom.xml +++ b/phoenix5-spark3/pom.xml @@ -66,12 +66,6 @@ ${spark.version} provided - - org.apache.spark - spark-tags_${scala.binary.version} - ${spark.version} - provided - org.apache.spark spark-catalyst_${scala.binary.version} @@ -137,18 +131,8 @@ hadoop-common provided - - org.apache.hadoop - hadoop-mapreduce-client-core - provided - - + - - joda-time - joda-time - ${jodatime.version} - @@ -270,6 +254,10 @@ + + org.apache.maven.plugins + maven-failsafe-plugin + org.scalatest scalatest-maven-plugin @@ -289,7 +277,7 @@ test - false Integration-Test 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 64fb0f5b..45ea0651 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 @@ -73,109 +73,97 @@ public Configuration getConfiguration(Configuration confToClone) { @Test public void basicWriteAndReadBackTest() throws SQLException { - SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkUtil.getSparkSession(); String tableName = generateUniqueName(); - try (Connection conn = DriverManager.getConnection(getUrl()); Statement stmt = conn.createStatement()) { stmt.executeUpdate( "CREATE TABLE " + tableName + " (id INTEGER PRIMARY KEY, v1 VARCHAR)"); } + StructType schema = + new StructType(new StructField[] { + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("v1", DataTypes.StringType, false, Metadata.empty()) }); + + // Use old zkUrl + Dataset df1 = + spark.createDataFrame( + Arrays.asList(RowFactory.create(1, "x")), + schema); - try (SparkSession spark = sqlContext.sparkSession()) { - - StructType schema = - new StructType(new StructField[] { - new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), - new StructField("v1", DataTypes.StringType, false, Metadata.empty()) }); + df1.write().format("phoenix").mode(SaveMode.Append) + .option(PhoenixDataSource.TABLE, tableName) + .option(ZOOKEEPER_URL, getUrl()) + .save(); + + // Use jdbcUrl + // In Phoenix 5.2+ getUrl() return a JDBC URL, in earlier versions it returns a ZK + // quorum + String jdbcUrl = getUrl(); + if (!jdbcUrl.startsWith(JDBC_PROTOCOL)) { + jdbcUrl = JDBC_PROTOCOL_ZK + JDBC_PROTOCOL_SEPARATOR + jdbcUrl; + } - // Use old zkUrl - Dataset df1 = - spark.createDataFrame( - Arrays.asList(RowFactory.create(1, "x")), - schema); + Dataset df2 = + spark.createDataFrame( + Arrays.asList(RowFactory.create(2, "x")), + schema); - df1.write().format("phoenix").mode(SaveMode.Append) + df2.write().format("phoenix").mode(SaveMode.Append) .option(PhoenixDataSource.TABLE, tableName) - .option(ZOOKEEPER_URL, getUrl()) + .option(JDBC_URL, jdbcUrl) .save(); - // Use jdbcUrl - // In Phoenix 5.2+ getUrl() return a JDBC URL, in earlier versions it returns a ZK - // quorum - String jdbcUrl = getUrl(); - if (!jdbcUrl.startsWith(JDBC_PROTOCOL)) { - jdbcUrl = JDBC_PROTOCOL_ZK + JDBC_PROTOCOL_SEPARATOR + jdbcUrl; - } - - Dataset df2 = - spark.createDataFrame( - Arrays.asList(RowFactory.create(2, "x")), - schema); + // Use default from hbase-site.xml + Dataset df3 = + spark.createDataFrame( + Arrays.asList(RowFactory.create(3, "x")), + schema); - df2.write().format("phoenix").mode(SaveMode.Append) - .option(PhoenixDataSource.TABLE, tableName) - .option(JDBC_URL, jdbcUrl) - .save(); + df3.write().format("phoenix").mode(SaveMode.Append) + .option(PhoenixDataSource.TABLE, tableName) + .save(); - // Use default from hbase-site.xml - Dataset df3 = - spark.createDataFrame( - Arrays.asList(RowFactory.create(3, "x")), - schema); + try (Connection conn = DriverManager.getConnection(getUrl()); + Statement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName); + assertTrue(rs.next()); + assertEquals(1, rs.getInt(1)); + assertEquals("x", rs.getString(2)); + assertTrue(rs.next()); + assertEquals(2, rs.getInt(1)); + assertEquals("x", rs.getString(2)); + assertTrue(rs.next()); + assertEquals(3, rs.getInt(1)); + assertEquals("x", rs.getString(2)); + assertFalse(rs.next()); + } - df3.write().format("phoenix").mode(SaveMode.Append) + Dataset df1Read = spark.read().format("phoenix") .option(PhoenixDataSource.TABLE, tableName) - .save(); + .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); - try (Connection conn = DriverManager.getConnection(getUrl()); - Statement stmt = conn.createStatement()) { - ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName); - assertTrue(rs.next()); - assertEquals(1, rs.getInt(1)); - assertEquals("x", rs.getString(2)); - assertTrue(rs.next()); - assertEquals(2, rs.getInt(1)); - assertEquals("x", rs.getString(2)); - assertTrue(rs.next()); - assertEquals(3, rs.getInt(1)); - assertEquals("x", rs.getString(2)); - assertFalse(rs.next()); - } - - Dataset df1Read = spark.read().format("phoenix") - .option(PhoenixDataSource.TABLE, tableName) - .option(PhoenixDataSource.JDBC_URL, getUrl()).load(); - - assertEquals(3l, df1Read.count()); - - // Use jdbcUrl - Dataset df2Read = spark.read().format("phoenix") - .option(PhoenixDataSource.TABLE, tableName) - .option(PhoenixDataSource.JDBC_URL, jdbcUrl) - .load(); + assertEquals(3l, df1Read.count()); - assertEquals(3l, df2Read.count()); + // Use jdbcUrl + Dataset df2Read = spark.read().format("phoenix") + .option(PhoenixDataSource.TABLE, tableName) + .option(PhoenixDataSource.JDBC_URL, jdbcUrl) + .load(); - // Use default - Dataset df3Read = spark.read().format("phoenix") - .option(PhoenixDataSource.TABLE, tableName) - .load(); + assertEquals(3l, df2Read.count()); - assertEquals(3l, df3Read.count()); + // Use default + Dataset df3Read = spark.read().format("phoenix") + .option(PhoenixDataSource.TABLE, tableName) + .load(); - } finally { - jsc.stop(); - } + assertEquals(3l, df3Read.count()); } @Test public void lowerCaseWriteTest() throws SQLException { - SparkConf sparkConf = new SparkConf().setMaster("local").setAppName("phoenix-test"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkUtil.getSparkSession(); String tableName = generateUniqueName(); try (Connection conn = DriverManager.getConnection(getUrl()); @@ -183,42 +171,33 @@ public void lowerCaseWriteTest() throws SQLException { stmt.executeUpdate("CREATE TABLE " + tableName + " (id INTEGER PRIMARY KEY, v1 VARCHAR, \"v1\" VARCHAR)"); } - try(SparkSession spark = sqlContext.sparkSession()) { - //Doesn't help - spark.conf().set("spark.sql.caseSensitive", true); - - StructType schema = new StructType(new StructField[]{ - new StructField("ID", DataTypes.IntegerType, false, Metadata.empty()), - new StructField("V1", DataTypes.StringType, false, Metadata.empty()), - new StructField("\"v1\"", DataTypes.StringType, false, Metadata.empty()) - }); - - Dataset df = spark.createDataFrame( - Arrays.asList( - RowFactory.create(1, "x", "y")), - schema); + StructType schema = new StructType(new StructField[]{ + new StructField("ID", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("V1", DataTypes.StringType, false, Metadata.empty()), + new StructField("\"v1\"", DataTypes.StringType, false, Metadata.empty()) + }); - df.write() - .format("phoenix") - .mode(SaveMode.Append) - .option(PhoenixDataSource.TABLE, tableName) - .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER,"true") - .option(JDBC_URL, getUrl()) - .save(); - - try (Connection conn = DriverManager.getConnection(getUrl()); - Statement stmt = conn.createStatement()) { - ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName); - assertTrue(rs.next()); - assertEquals(1, rs.getInt(1)); - assertEquals("x", rs.getString(2)); - assertEquals("y", rs.getString(3)); - assertFalse(rs.next()); - } + Dataset df = spark.createDataFrame( + Arrays.asList( + RowFactory.create(1, "x", "y")), + schema); + df.write() + .format("phoenix") + .mode(SaveMode.Append) + .option(PhoenixDataSource.TABLE, tableName) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER,"true") + .option(JDBC_URL, getUrl()) + .save(); - } finally { - jsc.stop(); + try (Connection conn = DriverManager.getConnection(getUrl()); + Statement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName); + assertTrue(rs.next()); + assertEquals(1, rs.getInt(1)); + assertEquals("x", rs.getString(2)); + assertEquals("y", rs.getString(3)); + assertFalse(rs.next()); } } 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 f217e46b..01e20fa3 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 @@ -67,7 +67,7 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter lazy val jdbcUrl = PhoenixSparkITHelper.getUrl - lazy val quorumAddress = ConfigurationUtil.getZookeeperURL(hbaseConfiguration).get + lazy val quorumAddress = PhoenixSparkITHelper.getUrl def getZookeeperURL(conf: Configuration): Option[String] = { List( diff --git a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala deleted file mode 100644 index 1251b858..00000000 --- a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1IT.scala +++ /dev/null @@ -1,737 +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.mapreduce.util.PhoenixConfigurationUtil -import org.apache.phoenix.schema.types.{PSmallintArray, PVarchar} -import org.apache.phoenix.spark.sql.connector.reader.PhoenixTestPartitionReader -import org.apache.phoenix.spark.sql.connector.writer.PhoenixTestBatchWrite -import org.apache.phoenix.spark.sql.connector.{PhoenixDataSource, PhoenixTestingDataSource} -import org.apache.phoenix.util.{ColumnInfo, SchemaUtil} -import org.apache.spark.SparkException -import org.apache.spark.sql.types._ -import org.apache.spark.sql.{Row, SaveMode} - -import java.sql.DriverManager -import java.time.{LocalDate, ZoneId} -import scala.collection.mutable -import scala.collection.mutable.ListBuffer - -/** - * Note: If running directly from an IDE, these are the recommended VM parameters: - * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m - */ -class PhoenixSparkDatasourceV1IT extends AbstractPhoenixSparkIT { - - test("Can persist data with case sensitive columns (like in avro schema)") { - val df = spark.createDataFrame( - Seq( - (1, 1, "test_child_1"), - (2, 1, "test_child_2"))). - // column names are case sensitive - toDF("ID", "TABLE3_ID", "t2col1") - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "TABLE3", - PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, - PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true")) - .mode(SaveMode.Overwrite) - .save() - - - // Verify results - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT * FROM TABLE3") - - val checkResults = List((1, 1, "test_child_1"), (2, 1, "test_child_2")) - val results = ListBuffer[(Long, Long, String)]() - while (rs.next()) { - results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) - } - stmt.close() - - results.toList shouldEqual checkResults - } - - test("Can convert Phoenix schema") { - val phoenixSchema = List( - new ColumnInfo("varcharColumn", PVarchar.INSTANCE.getSqlType) - ) - - val catalystSchema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(phoenixSchema) - - val expected = new StructType(List(StructField("varcharColumn", StringType, nullable = true)).toArray) - - catalystSchema shouldEqual expected - } - - test("Can convert arrays of Short type in Phoenix schema") { - val phoenixSchema = List( - new ColumnInfo("arrayshortColumn", PSmallintArray.INSTANCE.getSqlType) - ) - - val catalystSchema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(phoenixSchema) - - val expected = new StructType(List(StructField("arrayshortColumn", ArrayType(ShortType, true), nullable = true)).toArray) - - catalystSchema shouldEqual expected - } - - test("Can create schema RDD and execute query") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df1.createOrReplaceTempView("sql_table_1") - - val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df2.createOrReplaceTempView("sql_table_2") - - val sqlRdd = spark.sql( - """ - |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 - |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin - ) - - val count = sqlRdd.count() - - count shouldEqual 6L - } - - ignore("Ordering by pk columns should not require sorting") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - df1.createOrReplaceTempView("TABLE1") - - val sqlRdd = spark.sql("SELECT * FROM TABLE1 ORDER BY ID, COL1") - val plan = sqlRdd.queryExecution.sparkPlan - // verify the spark plan doesn't have a sort - assert(!plan.toString.contains("Sort")) - - val expectedResults = Array(Row.fromSeq(Seq(1, "test_row_1")), Row.fromSeq(Seq(2, "test_row_2"))) - val actual = sqlRdd.collect() - - actual shouldEqual expectedResults - } - - test("Verify correct number of partitions are created") { - val conn = DriverManager.getConnection(PhoenixSparkITHelper.getUrl) - val ddl = "CREATE TABLE SPLIT_TABLE (id VARCHAR NOT NULL PRIMARY KEY, val VARCHAR) split on ('e','j','o')" - conn.createStatement.execute(ddl) - val keys = Array("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s", - "t", "u", "v", "w", "x", "y", "z") - for (key <- keys) { - conn.createStatement.execute("UPSERT INTO SPLIT_TABLE VALUES('" + key + "', '" + key + "')") - } - conn.commit() - - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "SPLIT_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - df1.createOrReplaceTempView("SPLIT_TABLE") - val sqlRdd = spark.sql("SELECT * FROM SPLIT_TABLE") - val numPartitions = sqlRdd.rdd.partitions.size - - numPartitions shouldEqual 4 - } - - test("Can create schema RDD and execute query on case sensitive table (no config)") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("table4"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df1.createOrReplaceTempView("table4") - - val sqlRdd = spark.sql("SELECT id FROM table4") - - val count = sqlRdd.count() - - count shouldEqual 2L - } - - test("Can create schema RDD and execute constrained query") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df1.createOrReplaceTempView("sql_table_1") - - val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = 1") - - df2.createOrReplaceTempView("sql_table_2") - - val sqlRdd = spark.sql( - """ - |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 - |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin - ) - - val count = sqlRdd.count() - - count shouldEqual 1L - } - - test("Can create schema RDD with predicate that will never match") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = -1") - - df1.createOrReplaceTempView("table3") - - val sqlRdd = spark.sql("SELECT * FROM table3") - - val count = sqlRdd.count() - - count shouldEqual 0L - } - - test("Can create schema RDD with complex predicate") { - val predicate = "ID > 0 AND TIMESERIES_KEY BETWEEN " + - "CAST(TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP) AND " + - "CAST(TO_DATE('1990-01-30 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP)" - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "DATE_PREDICATE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - .filter(predicate) - - df1.createOrReplaceTempView("date_predicate_test_table") - - val sqlRdd = spark.sqlContext.sql("SELECT * FROM date_predicate_test_table") - - val count = sqlRdd.count() - - count shouldEqual 0L - } - - test("Can query an array table") { - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - df1.createOrReplaceTempView("ARRAY_TEST_TABLE") - - val sqlRdd = spark.sql("SELECT * FROM ARRAY_TEST_TABLE") - - val count = sqlRdd.count() - - // get row 0, column 1, which should be "VCARRAY" - val arrayValues = sqlRdd.collect().apply(0).apply(1) - - arrayValues should equal(Array("String1", "String2", "String3")) - - count shouldEqual 1L - } - - test("Can read a table as an RDD") { - val rdd1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - val count = rdd1.count() - - val arrayValues = rdd1.take(1)(0)(1) - - arrayValues should equal(Array("String1", "String2", "String3")) - - count shouldEqual 1L - } - - test("Can save to phoenix table from Spark without specifying all the columns") { - val dataSet = List(Row(1L, "1", 1), Row(2L, "2", 2), Row(3L, "3", 3)) - - // COL3 is missing both from the schema and from the dataset - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("COL1", StringType), - StructField("COL2", IntegerType))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT ID, COL1, COL2 FROM OUTPUT_TEST_TABLE") - val results = ListBuffer[Row]() - while (rs.next()) { - results.append(Row(rs.getLong(1), rs.getString(2), rs.getInt(3))) - } - - // Verify they match - (0 to results.size - 1).foreach { i => - dataSet(i) shouldEqual results(i) - } - } - - test("Can save dates to Phoenix using java.sql.Date") { - val date = java.sql.Date.valueOf("2016-09-30") - - // Since we are creating a Row we have to use java.sql.date - // java.util.date or joda.DateTime is not supported - val dataSet = Seq(Row(1L, "1", 1, date), Row(2L, "2", 2, date)) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("COL1", StringType), - StructField("COL2", IntegerType), - StructField("COL3", DateType))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT COL3 FROM OUTPUT_TEST_TABLE WHERE ID = 1 OR ID = 2 ORDER BY ID ASC") - val results = ListBuffer[java.sql.Date]() - while (rs.next()) { - results.append(rs.getDate(1)) - } - - // Verify the epochs are equal - results(0).getTime shouldEqual date.getTime - results(1).getTime shouldEqual date.getTime - } - - test("Can infer schema without defining columns") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - df.schema("ID").dataType shouldEqual LongType - df.schema("TABLE1_ID").dataType shouldEqual LongType - df.schema("t2col1").dataType shouldEqual StringType - } - - test("Spark SQL can use Phoenix as a data source with no schema specified") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - df.count() shouldEqual 2 - df.schema("ID").dataType shouldEqual LongType - df.schema("COL1").dataType shouldEqual StringType - } - - test("Can persist a dataframe") { - // Load from TABLE1 - val df = spark.sqlContext.read.format("org.apache.phoenix.spark").options( Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load - - // Save to TABLE1_COPY - df - .write - .format("org.apache.phoenix.spark") - .mode(SaveMode.Overwrite) - .option(PhoenixDataSource.TABLE, "TABLE1_COPY") - .option(PhoenixDataSource.ZOOKEEPER_URL, quorumAddress) - .save() - - // Verify results - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT * FROM TABLE1_COPY") - - val checkResults = List((1L, "test_row_1"), (2, "test_row_2")) - val results = ListBuffer[(Long, String)]() - while (rs.next()) { - results.append((rs.getLong(1), rs.getString(2))) - } - stmt.close() - - results.toList shouldEqual checkResults - } - - test("Can save arrays back to phoenix") { - val dataSet = List(Row(2L, Array("String1", "String2", "String3"))) - val schema = StructType(Seq( - StructField("ID", LongType, nullable = false), - StructField("VCARRAY", ArrayType(StringType, true)) - )) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT VCARRAY FROM ARRAY_TEST_TABLE WHERE ID = 2") - rs.next() - val sqlArray = rs.getArray(1).getArray().asInstanceOf[Array[String]] - - // Verify the arrays are equal - sqlArray shouldEqual dataSet(0).get(1) - } - - test("Can read from table with schema and escaped table name") { - // Manually escape - val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - - var count = df1.count() - - count shouldEqual 1L - - // Use SchemaUtil - val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options( - Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load() - - count = df2.count() - - count shouldEqual 1L - } - - test("Ensure DataFrame field normalization (PHOENIX-2196)") { - val rdd1 = spark.sparkContext - .parallelize(Seq((1L, 1L, "One"), (2L, 2L, "Two"))) - .map(p => Row(p._1, p._2, p._3)) - - val schema = StructType(Seq( - StructField("id", LongType, nullable = false), - StructField("table1_id", LongType, nullable = true), - StructField("\"t2col1\"", StringType, nullable = true) - )) - - val df = spark.sqlContext.createDataFrame(rdd1, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - } - - test("Ensure Dataframe supports LIKE and IN filters (PHOENIX-2328)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark").options(Map(PhoenixDataSource.TABLE -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - // Prefix match - val res1 = df.filter("COL1 like 'test_row_%'") - val plan = res1.groupBy().count().queryExecution.sparkPlan - res1.count() shouldEqual 2 - - // Suffix match - val res2 = df.filter("COL1 like '%_1'") - res2.count() shouldEqual 1 - res2.first.getString(1) shouldEqual "test_row_1" - - // Infix match - val res3 = df.filter("COL1 like '%_row_%'") - res3.count() shouldEqual 2 - - // Not like, match none - val res4 = df.filter("COL1 not like '%_row_%'") - res4.count() shouldEqual 0 - - // Not like, match all - val res5 = df.filter("COL1 not like '%_wor_%'") - res5.count() shouldEqual 2 - - // "IN", match all - val res6 = df.filter("COL1 in ('test_row_1', 'test_row_2')") - res6.count() shouldEqual 2 - - // "IN", match none - val res7 = df.filter("COL1 in ('foo', 'bar')") - res7.count() shouldEqual 0 - - // AND (and not again) - val res8 = df.filter("COL1 like '%_row_%' AND COL1 not like '%_1'") - res8.count() shouldEqual 1 - res8.first.getString(1) shouldEqual "test_row_2" - - // OR - val res9 = df.filter("COL1 like '%_1' OR COL1 like '%_2'") - res9.count() shouldEqual 2 - } - - test("Can load decimal types with accurate precision and scale (PHOENIX-2288)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "TEST_DECIMAL", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - assert(df.select("COL1").first().getDecimal(0) == BigDecimal("123.456789").bigDecimal) - } - - test("Can load small and tiny integer types (PHOENIX-2426)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "TEST_SMALL_TINY", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load() - assert(df.select("COL1").first().getShort(0).toInt == 32767) - assert(df.select("COL2").first().getByte(0).toInt == 127) - } - - test("Can save arrays from custom dataframes back to phoenix") { - val dataSet = List(Row(2L, Array("String1", "String2", "String3"), Array(1, 2, 3))) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("VCARRAY", ArrayType(StringType)), - StructField("INTARRAY", ArrayType(IntegerType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "ARRAYBUFFER_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT VCARRAY, INTARRAY FROM ARRAYBUFFER_TEST_TABLE WHERE ID = 2") - rs.next() - val stringArray = rs.getArray(1).getArray().asInstanceOf[Array[String]] - val intArray = rs.getArray(2).getArray().asInstanceOf[Array[Int]] - - // Verify the arrays are equal - stringArray shouldEqual dataSet(0).getAs[Array[String]](1) - intArray shouldEqual dataSet(0).getAs[Array[Int]](2) - } - - test("Can save arrays of AnyVal type back to phoenix") { - val dataSet = List(Row(2L, Array(1, 2, 3), Array(1L, 2L, 3L))) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("INTARRAY", ArrayType(IntegerType)), - StructField("BIGINTARRAY", ArrayType(LongType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "ARRAY_ANYVAL_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT INTARRAY, BIGINTARRAY FROM ARRAY_ANYVAL_TEST_TABLE WHERE ID = 2") - rs.next() - val intArray = rs.getArray(1).getArray().asInstanceOf[Array[Int]] - val longArray = rs.getArray(2).getArray().asInstanceOf[Array[Long]] - - // Verify the arrays are equal - intArray shouldEqual dataSet(0).get(1) - longArray shouldEqual dataSet(0).get(2) - } - - test("Can save arrays of Byte type back to phoenix") { - val dataSet = List(Row(2L, Array(1.toByte, 2.toByte, 3.toByte))) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("BYTEARRAY", ArrayType(ByteType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "ARRAY_BYTE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT BYTEARRAY FROM ARRAY_BYTE_TEST_TABLE WHERE ID = 2") - rs.next() - val byteArray = rs.getArray(1).getArray().asInstanceOf[Array[Byte]] - - // Verify the arrays are equal - byteArray shouldEqual dataSet(0).get(1) - } - - test("Can save arrays of Short type back to phoenix") { - val dataSet = List(Row(2L, Array(1.toShort, 2.toShort, 3.toShort))) - - val schema = StructType( - Seq(StructField("ID", LongType, nullable = false), - StructField("SHORTARRAY", ArrayType(ShortType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "ARRAY_SHORT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT SHORTARRAY FROM ARRAY_SHORT_TEST_TABLE WHERE ID = 2") - rs.next() - val shortArray = rs.getArray(1).getArray().asInstanceOf[Array[Short]] - - // Verify the arrays are equal - shortArray shouldEqual dataSet(0).get(1) - } - - test("Can save binary types back to phoenix") { - val dataSet = List(Row(2L, Array[Byte](1), Array[Byte](1, 2, 3), Array[Array[Byte]](Array[Byte](1), Array[Byte](2)))) - - val schema = StructType( - Seq(StructField("ID", LongType, false), - StructField("BIN", BinaryType), - StructField("VARBIN", BinaryType), - StructField("BINARRAY", ArrayType(BinaryType)))) - - val rowRDD = spark.sparkContext.parallelize(dataSet) - - // Apply the schema to the RDD. - val df = spark.sqlContext.createDataFrame(rowRDD, schema) - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "VARBINARY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - // Load the results back - val stmt = conn.createStatement() - val rs = stmt.executeQuery("SELECT BIN, VARBIN, BINARRAY FROM VARBINARY_TEST_TABLE WHERE ID = 2") - rs.next() - val byte = rs.getBytes("BIN") - val varByte = rs.getBytes("VARBIN") - val varByteArray = rs.getArray("BINARRAY").getArray().asInstanceOf[Array[Array[Byte]]] - - // Verify the arrays are equal - byte shouldEqual dataSet(0).get(1) - varByte shouldEqual dataSet(0).get(2) - varByteArray shouldEqual dataSet(0).get(3) - } - - test("Can load and filter Phoenix DATE columns through DataFrame API") { - val df = spark.sqlContext.read - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - val dt = df.select("COL1").first().getDate(0).getTime - val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - - assert(expected == dt) - - df.createOrReplaceTempView("DATE_TEST") - val df2 = spark.sql("SELECT * FROM DATE_TEST WHERE COL1 > TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss')") - assert(df2.count() == 1L) - } - - test("Filter operation doesn't work for column names containing a white space (PHOENIX-2547)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("space"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - val res = df.filter(df.col("first name").equalTo("xyz")) - // Make sure we got the right value back - assert(res.collectAsList().size() == 1L) - } - - test("Spark Phoenix cannot recognize Phoenix view fields (PHOENIX-2290)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - df.createOrReplaceTempView("temp") - - // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions - // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle - - val res1 = spark.sql("select * from temp where salary = '10000' ") - assert(res1.collectAsList().size() == 1L) - - val res2 = spark.sql("select * from temp where \"salary\" = '10000' ") - assert(res2.collectAsList().size() == 0L) - - val res3 = spark.sql("select * from temp where salary > '10000' ") - assert(res3.collectAsList().size() == 2L) - } - - test("Queries with small case column-names return empty result-set when working with Spark Datasource Plugin (PHOENIX-2336)") { - val df = spark.sqlContext.read.format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - - // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions - // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle - - val res1 = df.filter(df.col("first name").equalTo("foo")) - assert(res1.collectAsList().size() == 1L) - - val res2 = df.filter("\"first name\" = 'foo'") - assert(res2.collectAsList().size() == 0L) - - val res3 = df.filter("salary = '10000'") - assert(res3.collectAsList().size() == 1L) - - val res4 = df.filter("salary > '10000'") - assert(res4.collectAsList().size() == 2L) - } - - test("Can coerce Phoenix DATE columns to TIMESTAMP through DataFrame API") { - val df = spark.sqlContext.read - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, PhoenixDataSource.DATE_AS_TIME_STAMP -> "true")) - .load - val dtRes = df.select("COL1").first() - val ts = dtRes.getTimestamp(0).getTime - val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - - assert(expected == ts) - } - - test("Can load Phoenix Time columns through DataFrame API") { - val df = spark.sqlContext.read - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "TIME_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - val time = df.select("COL1").first().getTimestamp(0).getTime - val expected = LocalDate.of(2021, 1, 1).atStartOfDay(ZoneId.systemDefault()).toInstant.toEpochMilli - - assert(expected == time) - } - - test("can read all Phoenix data types") { - val df = spark.sqlContext.read - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .load - - df.write - .format("org.apache.phoenix.spark") - .options(Map(PhoenixDataSource.TABLE -> "OUTPUT_GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)) - .mode(SaveMode.Overwrite) - .save() - - df.count() shouldEqual 1 - } - -} diff --git a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1IT.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1IT.scala new file mode 100644 index 00000000..a019a039 --- /dev/null +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1IT.scala @@ -0,0 +1,150 @@ +/* + 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.rdd.RDD +import org.apache.spark.sql.SaveMode + +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 PhoenixSparkV1IT extends AbstractPhoenixSparkIT { + + + test("Can persist data with case sensitive columns (like in avro schema)") { + val df = spark.createDataFrame( + Seq( + (1, 1, "test_child_1"), + (2, 1, "test_child_2"))). + // column names are case sensitive + toDF("ID", "TABLE3_ID", "t2col1") + + df. + write + .format("org.apache.phoenix.spark") + .option(PhoenixDataSource.TABLE,"TABLE3") + .option(PhoenixDataSource.JDBC_URL,jdbcUrl) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER,"true") + .mode(SaveMode.Overwrite) + .save() + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3") + + val checkResults = List((1, 1, "test_child_1"), (2, 1, "test_child_2")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + + test("Can persist data with case sensitive columns using utility method") { + val df = spark.createDataFrame( + Seq( + (1, 1, "test_child_1"), + (2, 1, "test_child_2"))). + // column names are case sensitive + toDF("ID", "TABLE3_ID", "t2col1") + df + .saveToPhoenix(tableName = "TABLE3", + zkUrl = Some(jdbcUrl), + conf = hbaseConfiguration, + skipNormalizingIdentifier = true) + + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE3") + + val checkResults = List((1, 1, "test_child_1"), (2, 1, "test_child_2")) + val results = ListBuffer[(Long, Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getLong(2), rs.getString(3))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + + test("Can create schema RDD and execute query") { + val df1 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df1.createOrReplaceTempView("sql_table_1") + + val df2 = spark.sqlContext.read.format("org.apache.phoenix.spark") + .options(Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load + + df2.createOrReplaceTempView("sql_table_2") + + val sqlRdd = spark.sql( + """ + |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1 + |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin + ) + + val count = sqlRdd.count() + + count shouldEqual 6L + } + + test("can read selected columns As RDD with filter") { + val expected: Seq[Map[String, AnyRef]] = Seq( + Map( + "ID" -> 1.asInstanceOf[AnyRef], + "T2COL1" -> "test_child_1".asInstanceOf[AnyRef] + ) + ) + val rdd: RDD[Map[String, AnyRef]] = spark.sparkContext.phoenixTableAsRDD( + table = "TABLE2", + columns = Seq("ID", "T2COL1"), + zkUrl = Some(quorumAddress), + conf = hbaseConfiguration, + predicate = Some("ID = 1")) + + rdd.collect().toSeq shouldEqual expected + } + + test("Can persist a dataframe") { + // Load from TABLE1 + val df = spark.sqlContext.phoenixTableAsDataFrame( + "TABLE1", + Nil, + zkUrl = Some(quorumAddress), + conf = hbaseConfiguration) + // Save to TABLE1_COPY + df + .saveToPhoenix(tableName = "TABLE1_COPY", zkUrl = Some(quorumAddress), conf = hbaseConfiguration) + // Verify results + val stmt = conn.createStatement() + val rs = stmt.executeQuery("SELECT * FROM TABLE1_COPY") + + val checkResults = List((1L, "test_row_1"), (2, "test_row_2")) + val results = ListBuffer[(Long, String)]() + while (rs.next()) { + results.append((rs.getLong(1), rs.getString(2))) + } + stmt.close() + + results.toList shouldEqual checkResults + } + +} diff --git a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1ITTenantSpecific.scala similarity index 97% rename from phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala rename to phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1ITTenantSpecific.scala index 28907dbc..1daee5f2 100644 --- a/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkDatasourceV1ITTenantSpecific.scala +++ b/phoenix5-spark3/src/it/scala/org/apache/phoenix/spark/PhoenixSparkV1ITTenantSpecific.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ListBuffer * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m * */ -class PhoenixSparkDatasourceV1ITTenantSpecific extends AbstractPhoenixSparkIT { +class PhoenixSparkV1ITTenantSpecific extends AbstractPhoenixSparkIT { // Tenant-specific schema info val OrgIdCol = "ORGANIZATION_ID" 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 37f942ae..d538bd45 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 @@ -161,7 +161,8 @@ public String shortName() { } @Override - public BaseRelation createRelation(SQLContext sqlContext, scala.collection.immutable.Map parameters) { + public BaseRelation createRelation(SQLContext sqlContext, + scala.collection.immutable.Map parameters) { return new PhoenixSparkSqlRelation( sqlContext.sparkSession(), @@ -175,10 +176,10 @@ private List generateColumnInfo(Connection conn, String tableName) t PTable table = PhoenixRuntime.getTable(conn, SchemaUtil.normalizeFullTableName(tableName)); int startOffset = 0; - if(table.getTenantId()!=null) { + if (table.getTenantId() != null) { startOffset++; } - if(table.getBucketNum()!=null){ + if (table.getBucketNum() != null) { startOffset++; } 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 685c753a..304c4595 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 @@ -17,10 +17,6 @@ */ package org.apache.phoenix.spark.sql.connector; -import static org.apache.spark.sql.connector.catalog.TableCapability.ACCEPT_ANY_SCHEMA; -import static org.apache.spark.sql.connector.catalog.TableCapability.BATCH_READ; -import static org.apache.spark.sql.connector.catalog.TableCapability.BATCH_WRITE; - import java.util.Map; import java.util.Set; @@ -36,14 +32,19 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import static org.apache.spark.sql.connector.catalog.TableCapability.BATCH_READ; +import static org.apache.spark.sql.connector.catalog.TableCapability.BATCH_WRITE; +import static org.apache.spark.sql.connector.catalog.TableCapability.ACCEPT_ANY_SCHEMA; +import static org.apache.spark.sql.connector.catalog.TableCapability.OVERWRITE_BY_FILTER; public class PhoenixTable implements SupportsRead, SupportsWrite { 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); + private static final Set + CAPABILITIES = + ImmutableSet.of(BATCH_READ, BATCH_WRITE, ACCEPT_ANY_SCHEMA, OVERWRITE_BY_FILTER); public PhoenixTable(StructType schema, Map options) { this.options = options; diff --git a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/writer/PhoenixWriteBuilder.java b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/writer/PhoenixWriteBuilder.java index 32a595e2..0236aeb5 100644 --- a/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/writer/PhoenixWriteBuilder.java +++ b/phoenix5-spark3/src/main/java/org/apache/phoenix/spark/sql/connector/writer/PhoenixWriteBuilder.java @@ -20,11 +20,28 @@ import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.spark.sql.connector.write.BatchWrite; import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.SupportsOverwrite; import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.sources.Filter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Map; -public class PhoenixWriteBuilder implements WriteBuilder { +/** + * The PhoenixWriteBuilder class is responsible for constructing + * and configuring a write operation for Phoenix when interfacing + * with Spark's data source API. + * This class implements the WriteBuilder interface for write operations + * and SupportsOverwrite interface to handle overwrite behavior. + * The class facilitates the creation of a batch write operation + * that is configured with the provided logical writing information + * and options specific to the Phoenix data source. + * Note: Overwrite mode does not do truncate table + * and behaves the same as Append mode. + */ +public class PhoenixWriteBuilder implements WriteBuilder, SupportsOverwrite { + private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixWriteBuilder.class); private final LogicalWriteInfo writeInfo; private final Map options; @@ -48,4 +65,11 @@ LogicalWriteInfo getWriteInfo() { Map getOptions() { return options; } + + @Override + public WriteBuilder overwrite(Filter[] filters) { + LOGGER.info("Overwrite mode specified. Ignoring Overwrite and treating it as Append."); + return this; + } + } 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 2436122a..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala +++ /dev/null @@ -1,108 +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.HConstants -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 index 3b0289d2..c7526a5e 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala @@ -14,66 +14,37 @@ 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._ +import org.apache.phoenix.spark.PhoenixDataFrameHelper.phoenixConfig +import org.apache.phoenix.spark.sql.connector.PhoenixDataSource +import org.apache.spark.sql.{DataFrame, SaveMode} @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 + saveToPhoenix( + tableName = parameters("table"), + zkUrl = parameters.get("zkUrl"), + tenantId = parameters.get("TenantId"), + skipNormalizingIdentifier = parameters.contains("skipNormalizingIdentifier") ) } - def getFieldArray(skipNormalizingIdentifier: Boolean = false, data: DataFrame) = { - if (skipNormalizingIdentifier) { - data.schema.fieldNames.map(x => x) - } else { - data.schema.fieldNames.map(x => SchemaUtil.normalizeIdentifier(x)) - } + def saveToPhoenix(tableName: String, + conf: Configuration = new Configuration, + zkUrl: Option[String] = None, + tenantId: Option[String] = None, + skipNormalizingIdentifier: Boolean = false): Unit = { + data + .write + .format("phoenix") + .mode(SaveMode.Append) + .option(PhoenixDataSource.TABLE, tableName) + .option(PhoenixDataSource.JDBC_URL, zkUrl.orNull) + .option(PhoenixDataSource.TENANT_ID, tenantId.orNull) + .option(PhoenixDataSource.PHOENIX_CONFIGS, phoenixConfig(conf)) + .option(PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER, skipNormalizingIdentifier) + .save() + } + } 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 index 7260c1b4..4c7cbab8 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala @@ -17,45 +17,6 @@ */ package org.apache.phoenix.spark -import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider} -import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} +import org.apache.phoenix.spark.sql.connector.PhoenixDataSource -@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") - } - -} +class DefaultSource extends PhoenixDataSource diff --git a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixDataFrameHelper.scala b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixDataFrameHelper.scala new file mode 100644 index 00000000..c9874afd --- /dev/null +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixDataFrameHelper.scala @@ -0,0 +1,76 @@ +/* + * 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.phoenix.spark.sql.connector.PhoenixDataSource +import org.apache.spark.sql.{DataFrame, SparkSession} + +import scala.collection.JavaConverters.asScalaIteratorConverter +import scala.collection.mutable + +private[spark] object PhoenixDataFrameHelper { + + def createDataFrame(table: String, + zkUrl: Option[String] = None, + tenantId: Option[String] = None, + conf: Configuration)(implicit sparkSession: SparkSession): DataFrame = { + val sparkOptions = mutable.Map(PhoenixDataSource.TABLE -> table, + PhoenixDataSource.PHOENIX_CONFIGS -> phoenixConfig(conf)) + + if (zkUrl.isDefined) { + sparkOptions += (PhoenixDataSource.JDBC_URL -> zkUrl.get) + } + if (tenantId.isDefined) { + sparkOptions += (PhoenixDataSource.TENANT_ID -> tenantId.get) + } + + sparkSession + .read + .format("phoenix") + .options(sparkOptions) + .load() + } + + def phoenixConfig(conf: Configuration): String = { + conf + .iterator() + .asScala + .toSeq + .filter(c => (c.getValue != null && c.getValue.trim.nonEmpty) && !c.getValue.contains(",")) + .map(c => s"${c.getKey}=${c.getValue}") + .mkString(",") + } + + def withSelectExpr(columns: Seq[String], df: DataFrame): DataFrame = { + if (columns.nonEmpty) { + df.selectExpr(columns: _*) + } else { + df + } + } + + def withWhereCondition(predicate: Option[String], df: DataFrame): DataFrame = { + if (predicate.isDefined) { + df.where(predicate.get) + } else { + df + } + } + +} 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 c0af5c4a..00000000 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala +++ /dev/null @@ -1,148 +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 2edf5463..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, Filter, PrunedFilteredScan} -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 index b0735218..aa1e588f 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala @@ -14,51 +14,58 @@ 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.spark.sql.connector.PhoenixDataSource +import org.apache.phoenix.util.PhoenixRuntime import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, SparkSession} -import scala.collection.JavaConversions._ +import java.sql.{DriverManager, SQLException} +import java.util.Properties +import scala.collection.JavaConverters +import scala.collection.JavaConverters.{asScalaIteratorConverter, mapAsJavaMapConverter} +import scala.jdk.CollectionConverters.seqAsJavaListConverter +import scala.util.{Failure, Success, Try} @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) - } + def saveToPhoenix(tableName: String, + cols: Seq[String], + conf: Configuration = new Configuration, + zkUrl: Option[String] = None, + tenantId: Option[String] = None): Unit = { + val sparkSession: SparkSession = SparkSession.builder().config(data.sparkContext.getConf).getOrCreate() + val dsOptions = Map(PhoenixDataSource.JDBC_URL -> zkUrl.orNull).asJava + val jdbcUrl = PhoenixDataSource.getJdbcUrlFromOptions(dsOptions) + val confAsMap = conf.iterator().asScala.map(c => (c.getKey -> c.getValue)).toMap.asJava + val confToSet = new Properties() + confToSet.putAll(confAsMap) + if (tenantId.isDefined) { + confToSet.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId.get) } + val schema: StructType = catalystSchema(tableName, cols, jdbcUrl, confToSet) + val dataFrame = sparkSession.createDataFrame(data.map(Row.fromTuple), schema).selectExpr(cols: _*) + new DataFrameFunctions(dataFrame) + .saveToPhoenix( + tableName = tableName, + conf = conf, + zkUrl = zkUrl, + tenantId = tenantId + ) + } - // 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 - ) + private def catalystSchema(tableName: String, columnList: Seq[String], jdbcUrl: String, overriddenProps: Properties): StructType = { + Try(DriverManager.getConnection(jdbcUrl, overriddenProps)) match { + case Success(conn) => try { + val columnInfos = PhoenixRuntime.generateColumnInfo(conn, tableName, columnList.asJava) + val columnInfoSeq = JavaConverters.asScalaIteratorConverter(columnInfos.iterator).asScala.toSeq + SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoSeq) + } catch { + case e: SQLException => + throw new RuntimeException(e) + } finally if (conn != null) conn.close() + case Failure(e) => throw new RuntimeException(e) + } } + } \ 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 index 1b377abe..75b750b5 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala @@ -16,6 +16,7 @@ package org.apache.phoenix.spark import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession @deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)") class SparkContextFunctions(@transient val sc: SparkContext) extends Serializable { @@ -32,11 +33,25 @@ class SparkContextFunctions(@transient val sc: SparkContext) extends Serializabl 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) + 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]] = { + + val sparkSession = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataFrame = sparkSession.sqlContext.phoenixTableAsDataFrame( + table = table, + columns = columns, + predicate = predicate, + zkUrl = zkUrl, + tenantId = tenantId, + conf = conf + ) + + dataFrame + .rdd + .map(row => row.getValuesMap(row.schema.fieldNames)) } } \ 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 index f9154adf..dec7b987 100644 --- a/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala +++ b/phoenix5-spark3/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala @@ -18,7 +18,6 @@ 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 @@ -30,13 +29,13 @@ class SparkSqlContextFunctions(@transient val sqlContext: SQLContext) extends Se 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) + predicate: Option[String] = None, + zkUrl: Option[String] = None, + tenantId: Option[String] = None, + conf: Configuration = new Configuration): DataFrame = { + val df = PhoenixDataFrameHelper.createDataFrame(table, zkUrl, tenantId, conf)(sqlContext.sparkSession) + val dfWithSelectColumns = PhoenixDataFrameHelper.withSelectExpr(columns, df) + PhoenixDataFrameHelper.withWhereCondition(predicate, dfWithSelectColumns) } + } \ No newline at end of file