[
https://issues.apache.org/jira/browse/PHOENIX-3427?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15654779#comment-15654779
]
ASF GitHub Bot commented on PHOENIX-3427:
-----------------------------------------
Github user jmahonin commented on a diff in the pull request:
https://github.com/apache/phoenix/pull/221#discussion_r87457154
--- Diff:
phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala
---
@@ -16,19 +16,20 @@ 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.{ColumnInfoToStringEncoderDecoder,
PhoenixConfigurationUtil}
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil
import org.apache.spark.Logging
import org.apache.spark.rdd.RDD
+
import scala.collection.JavaConversions._
class ProductRDDFunctions[A <: Product](data: RDD[A]) extends Logging with
Serializable {
def saveToPhoenix(tableName: String, cols: Seq[String],
- conf: Configuration = new Configuration, zkUrl:
Option[String] = None)
+ conf: Configuration = new Configuration, zkUrl:
Option[String] = None, tenantId: Option[String] = None)
--- End diff --
Ok taking a bit of a deeper dive, I think setting it directly on the
Configuration object is about the best we can do here. Those objects are
created on every save operation, so I don't think there's any possibility of
bleeding across multiple users.
In a bit more detail, the limitation is that from within the phoenix-spark
module, we don't have direct visibility or control over the Connection object.
We essentially just tell Spark to use the `PhoenixOutputFormat` and to use
`outConfig` as the Hadoop Configuration:
https://github.com/apache/phoenix/blob/master/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala#L51-L56
Eventually, Spark will invoke `phoenix.mapreduce.PhoenixRecordWriter`,
which will create the connection based on the configuration we've given it
above:
https://github.com/apache/phoenix/blob/master/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala#L51-L56
Following that call-path, the Connection object is created using a
Properties object that's derived from the initial Configuration:
https://github.com/apache/phoenix/blob/master/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/ConnectionUtil.java#L85
https://github.com/apache/phoenix/blob/master/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java#L51-L60
In effect, the Configuration is really the *only* interface to configuring
the Connection object at all.
> rdd.saveToPhoenix gives table undefined error when attempting to write to a
> tenant-specific view (TenantId defined in configuration object and passed to
> saveToPhoenix)
> -----------------------------------------------------------------------------------------------------------------------------------------------------------------------
>
> Key: PHOENIX-3427
> URL: https://issues.apache.org/jira/browse/PHOENIX-3427
> Project: Phoenix
> Issue Type: Bug
> Reporter: Nico Pappagianis
>
> Although we can read from a tenant-specific view by passing TenantId in the
> conf object when calling sc.phoenixTableAsRDD the same does not hold for
> rdd.saveToPhoenix. Calling saveToPhoenix with a tenant-specific view as the
> table name gives a table undefined error, even when passing in the TenantId
> with the conf object.
> It appears that TenantId is lost during the execution path of saveToPhoenix.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)