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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to