FYI, Nick - do you know about Josh's fix for PHOENIX-2599? Does that help here?
On Fri, Jan 22, 2016 at 4:32 PM, Nick Dimiduk <[email protected]> wrote: > On Thu, Jan 21, 2016 at 7:36 AM, Josh Mahonin <[email protected]> wrote: > >> Amazing, good work. >> > > All I did was consume your code and configure my cluster. Thanks though :) > > FWIW, I've got a support case in with Hortonworks to get the phoenix-spark >> integration working out of the box. Assuming it gets resolved, that'll >> hopefully help keep these classpath-hell issues to a minimum going forward. >> > > That's fine for HWX customers, but it doesn't solve the general problem > community-side. Unless, of course, we assume the only Phoenix users are on > HDP. > > Interesting point re: PHOENIX-2535. Spark does offer builds for specific >> Hadoop versions, and also no Hadoop at all (with the assumption you'll >> provide the necessary JARs). Phoenix is pretty tightly coupled with its own >> HBase (and by extension, Hadoop) versions though... do you think it be >> possible to work around (2) if you locally added the HDP Maven repo and >> adjusted versions accordingly? I've had some success with that in other >> projects, though as I recall when I tried it with Phoenix I ran into a snag >> trying to resolve some private transitive dependency of Hadoop. >> > > I could specify Hadoop version and build Phoenix locally to remove the > issue. It would work for me because I happen to be packaging my own Phoenix > this week, but it doesn't help for the Apache releases. Phoenix _is_ > tightly coupled to HBase versions, but I don't think HBase versions are > that tightly coupled to Hadoop versions. We build HBase 1.x releases > against a long list of Hadoop releases as part of our usual build. I think > what HBase consumes re: Hadoop API's is pretty limited. > > Now that you have Spark working you can start hitting real bugs! If you >> haven't backported the full patchset, you might want to take a look at the >> phoenix-spark history [1], there's been a lot of churn there, especially >> with regards to the DataFrame API. >> > > Yeah, speaking of which... :) > > I find this integration is basically unusable when the underlying HBase > table partitions are in flux; i.e., when you have data being loaded > concurrent to query. Spark RDD's assume stable partitions, multiple queries > against a single DataFrame, or even a single query/DataFrame with lots of > underlying splits and not enough workers, will inevitably fail > with ConcurrentModificationException like [0]. > > I think in HBase/MR integration, we define split points for the job as > region boundaries initially, but then we're just using the regular API, so > repartitions are handled transiently by the client layer. I need to dig > into this Phoenix/Spark stuff to see if we can do anything similar here. > > Thanks again Josh, > -n > > [0]: > > java.lang.RuntimeException: java.util.ConcurrentModificationException > at > org.apache.phoenix.mapreduce.PhoenixInputFormat.getQueryPlan(PhoenixInputFormat.java:125) > at > org.apache.phoenix.mapreduce.PhoenixInputFormat.createRecordReader(PhoenixInputFormat.java:69) > at org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(NewHadoopRDD.scala:131) > at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:104) > at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:66) > at org.apache.phoenix.spark.PhoenixRDD.compute(PhoenixRDD.scala:57) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) > at > org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:70) > at > org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) > at org.apache.spark.scheduler.Task.run(Task.scala:70) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.util.ConcurrentModificationException > at java.util.Hashtable$Enumerator.next(Hashtable.java:1367) > at org.apache.hadoop.conf.Configuration.iterator(Configuration.java:2154) > at > org.apache.phoenix.util.PropertiesUtil.extractProperties(PropertiesUtil.java:52) > at > org.apache.phoenix.mapreduce.util.ConnectionUtil.getInputConnection(ConnectionUtil.java:60) > at > org.apache.phoenix.mapreduce.util.ConnectionUtil.getInputConnection(ConnectionUtil.java:45) > at > org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil.getSelectColumnMetadataList(PhoenixConfigurationUtil.java:278) > at > org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil.getSelectStatement(PhoenixConfigurationUtil.java:306) > at > org.apache.phoenix.mapreduce.PhoenixInputFormat.getQueryPlan(PhoenixInputFormat.java:113) > ... 32 more > > On Thu, Jan 21, 2016 at 12:41 AM, Nick Dimiduk <[email protected]> >> wrote: >> >>> I finally got to the bottom of things. There were two issues at play in >>> my particular environment. >>> >>> 1. An Ambari bug [0] means my spark-defaults.conf file was garbage. I >>> hardly thought of it when I hit the issue with MR job submission; its >>> impact on Spark was much more subtle. >>> >>> 2. YARN client version mismatch (Phoenix is compiled vs Apache 2.5.1 >>> while my cluster is running HDP's 2.7.1 build), per my earlier email. Once >>> I'd worked around (1), I was able to work around (2) by setting >>> "/usr/hdp/current/hadoop-yarn-client/*:/usr/hdp/current/phoenix-client/phoenix-client-spark.jar" >>> for both spark.driver.extraClassPath and spark.executor.extraClassPath. Per >>> the linked thread above, I believe this places the correct YARN client >>> version first in the classpath. >>> >>> With the above in place, I'm able to submit work vs the Phoenix tables >>> to the YARN cluster. Success! >>> >>> Ironically enough, I would not have been able to work around (2) if we >>> had PHOENIX-2535 in place. Food for thought in tackling that issue. It may >>> be worth while to ship a uberclient jar that is entirely without Hadoop (or >>> HBase) classes. I believe spark does this for Hadoop with their builds as >>> well. >>> >>> Thanks again for your help here Josh! I really appreciate it. >>> >>> [0]: https://issues.apache.org/jira/browse/AMBARI-14751 >>> >>> On Wed, Jan 20, 2016 at 2:23 PM, Nick Dimiduk <[email protected]> >>> wrote: >>> >>>> Well, I spoke too soon. It's working, but in local mode only. When I >>>> invoke `pyspark --master yarn` (or yarn-client), the submitted application >>>> goes from ACCEPTED to FAILED, with a NumberFormatException [0] in my >>>> container log. Now that Phoenix is on my classpath, I'm suspicious that the >>>> versions of YARN client libraries are incompatible. I found an old thread >>>> [1] with the same stack trace I'm seeing, similar conclusion. I tried >>>> setting spark.driver.extraClassPath and spark.executor.extraClassPath >>>> to >>>> /usr/hdp/current/hadoop-yarn-client:/usr/hdp/current/phoenix-client/phoenix-client-spark.jar >>>> but that appears to have no impact. >>>> >>>> [0]: >>>> 16/01/20 22:03:45 ERROR yarn.ApplicationMaster: Uncaught exception: >>>> java.lang.IllegalArgumentException: Invalid ContainerId: >>>> container_e07_1452901320122_0042_01_000001 >>>> at >>>> org.apache.hadoop.yarn.util.ConverterUtils.toContainerId(ConverterUtils.java:182) >>>> at >>>> org.apache.spark.deploy.yarn.YarnRMClient.getAttemptId(YarnRMClient.scala:93) >>>> at >>>> org.apache.spark.deploy.yarn.ApplicationMaster.run(ApplicationMaster.scala:85) >>>> at >>>> org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$main$1.apply$mcV$sp(ApplicationMaster.scala:574) >>>> at >>>> org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:66) >>>> at >>>> org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:65) >>>> at java.security.AccessController.doPrivileged(Native Method) >>>> at javax.security.auth.Subject.doAs(Subject.java:422) >>>> at >>>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1614) >>>> at >>>> org.apache.spark.deploy.SparkHadoopUtil.runAsSparkUser(SparkHadoopUtil.scala:65) >>>> at >>>> org.apache.spark.deploy.yarn.ApplicationMaster$.main(ApplicationMaster.scala:572) >>>> at >>>> org.apache.spark.deploy.yarn.ExecutorLauncher$.main(ApplicationMaster.scala:599) >>>> at >>>> org.apache.spark.deploy.yarn.ExecutorLauncher.main(ApplicationMaster.scala) >>>> Caused by: java.lang.NumberFormatException: For input string: "e07" >>>> at >>>> java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) >>>> at java.lang.Long.parseLong(Long.java:589) >>>> at java.lang.Long.parseLong(Long.java:631) >>>> at >>>> org.apache.hadoop.yarn.util.ConverterUtils.toApplicationAttemptId(ConverterUtils.java:137) >>>> at >>>> org.apache.hadoop.yarn.util.ConverterUtils.toContainerId(ConverterUtils.java:177) >>>> ... 12 more >>>> >>>> [1]: >>>> http://mail-archives.us.apache.org/mod_mbox/spark-user/201503.mbox/%3CCAAqMD1jSEvfyw9oUBymhZukm7f+WTDVZ8E6Zp3L4a9OBJ-hz=a...@mail.gmail.com%3E >>>> >>>> On Wed, Jan 20, 2016 at 1:29 PM, Josh Mahonin <[email protected]> >>>> wrote: >>>> >>>>> That's great to hear. Looking forward to the doc patch! >>>>> >>>>> On Wed, Jan 20, 2016 at 3:43 PM, Nick Dimiduk <[email protected]> >>>>> wrote: >>>>> >>>>>> Josh -- I deployed my updated phoenix build across the cluster, added >>>>>> the phoenix-client-spark.jar to configs on the whole cluster, and now >>>>>> basic >>>>>> dataframe access is now working. Let me see about updating the docs page >>>>>> to >>>>>> be more clear, I'll send a patch by you for review. >>>>>> >>>>>> Thanks a lot for the help! >>>>>> -n >>>>>> >>>>>> On Tue, Jan 19, 2016 at 5:59 PM, Josh Mahonin <[email protected]> >>>>>> wrote: >>>>>> >>>>>>> Right, this cluster I just tested on is HDP 2.3.4, so it's Spark on >>>>>>> YARN as well. I suppose the JAR is probably shipped by YARN, though I >>>>>>> don't >>>>>>> see any logging saying it, so I'm not certain how the nuts and bolts of >>>>>>> that work. By explicitly setting the classpath, we're bypassing Spark's >>>>>>> native JAR broadcast though. >>>>>>> >>>>>>> Taking a quick look at the config in Ambari (which ships the config >>>>>>> to each node after saving), in 'Custom spark-defaults' I have the >>>>>>> following: >>>>>>> >>>>>>> spark.driver.extraClassPath -> >>>>>>> /etc/hbase/conf:/usr/hdp/current/phoenix-client/phoenix-client-spark.jar >>>>>>> spark.executor.extraClassPath -> >>>>>>> /usr/hdp/current/phoenix-client/phoenix-client-spark.jar >>>>>>> >>>>>>> I'm not sure if the /etc/hbase/conf is necessarily needed, but I >>>>>>> think that gets the Ambari generated hbase-site.xml in the classpath. >>>>>>> Each >>>>>>> node has the custom phoenix-client-spark.jar installed to that same >>>>>>> path as >>>>>>> well. >>>>>>> >>>>>>> I can pop into regular spark-shell and load RDDs/DataFrames using: >>>>>>> /usr/hdp/current/spark-client/bin/spark-shell --master yarn-client >>>>>>> >>>>>>> or pyspark via: >>>>>>> /usr/hdp/current/spark-client/bin/pyspark >>>>>>> >>>>>>> I also do this as the Ambari-created 'spark' user, I think there was >>>>>>> some fun HDFS permission issue otherwise. >>>>>>> >>>>>>> On Tue, Jan 19, 2016 at 8:23 PM, Nick Dimiduk <[email protected]> >>>>>>> wrote: >>>>>>> >>>>>>>> I'm using Spark on YARN, not spark stand-alone. YARN NodeManagers >>>>>>>> are colocated with RegionServers; all the hosts have everything. There >>>>>>>> are >>>>>>>> no spark workers to restart. You're sure it's not shipped by the YARN >>>>>>>> runtime? >>>>>>>> >>>>>>>> On Tue, Jan 19, 2016 at 5:07 PM, Josh Mahonin <[email protected]> >>>>>>>> wrote: >>>>>>>> >>>>>>>>> Sadly, it needs to be installed onto each Spark worker (for now). >>>>>>>>> The executor config tells each Spark worker to look for that file to >>>>>>>>> add to >>>>>>>>> its classpath, so once you have it installed, you'll probably need to >>>>>>>>> restart all the Spark workers. >>>>>>>>> >>>>>>>>> I co-locate Spark and HBase/Phoenix nodes, so I just drop it in >>>>>>>>> /usr/hdp/current/phoenix-client/, but anywhere that each worker can >>>>>>>>> consistently see is fine. >>>>>>>>> >>>>>>>>> One day we'll be able to have Spark ship the JAR around and use it >>>>>>>>> without this classpath nonsense, but we need to do some extra work on >>>>>>>>> the >>>>>>>>> Phoenix side to make sure that Phoenix's calls to DriverManager >>>>>>>>> actually go >>>>>>>>> through Spark's weird wrapper version of it. >>>>>>>>> >>>>>>>>> On Tue, Jan 19, 2016 at 7:36 PM, Nick Dimiduk <[email protected] >>>>>>>>> > wrote: >>>>>>>>> >>>>>>>>>> On Tue, Jan 19, 2016 at 4:17 PM, Josh Mahonin <[email protected] >>>>>>>>>> > wrote: >>>>>>>>>> >>>>>>>>>>> What version of Spark are you using? >>>>>>>>>>> >>>>>>>>>> >>>>>>>>>> Probably HDP's Spark 1.4.1; that's what the jars in my install >>>>>>>>>> say, and the welcome message in the pyspark console agrees. >>>>>>>>>> >>>>>>>>>> Are there any other traces of exceptions anywhere? >>>>>>>>>>> >>>>>>>>>> >>>>>>>>>> No other exceptions that I can find. YARN apparently doesn't want >>>>>>>>>> to aggregate spark's logs. >>>>>>>>>> >>>>>>>>>> >>>>>>>>>>> Are all your Spark nodes set up to point to the same >>>>>>>>>>> phoenix-client-spark JAR? >>>>>>>>>>> >>>>>>>>>> >>>>>>>>>> Yes, as far as I can tell... though come to think of it, is that >>>>>>>>>> jar shipped by the spark runtime to workers, or is it loaded locally >>>>>>>>>> on >>>>>>>>>> each host? I only changed spark-defaults.conf on the client machine, >>>>>>>>>> the >>>>>>>>>> machine from which I submitted the job. >>>>>>>>>> >>>>>>>>>> Thanks for taking a look Josh! >>>>>>>>>> >>>>>>>>>> On Tue, Jan 19, 2016 at 5:02 PM, Nick Dimiduk < >>>>>>>>>>> [email protected]> wrote: >>>>>>>>>>> >>>>>>>>>>>> Hi guys, >>>>>>>>>>>> >>>>>>>>>>>> I'm doing my best to follow along with [0], but I'm hitting >>>>>>>>>>>> some stumbling blocks. I'm running with HDP 2.3 for HBase and >>>>>>>>>>>> Spark. My >>>>>>>>>>>> phoenix build is much newer, basically 4.6-branch + PHOENIX-2503, >>>>>>>>>>>> PHOENIX-2568. I'm using pyspark for now. >>>>>>>>>>>> >>>>>>>>>>>> I've added phoenix-$VERSION-client-spark.jar to both >>>>>>>>>>>> spark.executor.extraClassPath and spark.driver.extraClassPath. >>>>>>>>>>>> This allows >>>>>>>>>>>> me to use sqlContext.read to define a DataFrame against a Phoenix >>>>>>>>>>>> table. >>>>>>>>>>>> This appears to basically work, as I see PhoenixInputFormat in the >>>>>>>>>>>> logs and >>>>>>>>>>>> df.printSchema() shows me what I expect. However, when I try >>>>>>>>>>>> df.take(5), I >>>>>>>>>>>> get "IllegalStateException: unread block data" [1] from the >>>>>>>>>>>> workers. Poking >>>>>>>>>>>> around, this is commonly a problem with classpath. Any ideas as to >>>>>>>>>>>> specifically which jars are needed? Or better still, how to debug >>>>>>>>>>>> this >>>>>>>>>>>> issue myself. Adding "/usr/hdp/current/hbase-client/lib/*" to the >>>>>>>>>>>> classpath >>>>>>>>>>>> gives me a VerifyError about netty method version mismatch. Indeed >>>>>>>>>>>> I see >>>>>>>>>>>> two netty versions in that lib directory... >>>>>>>>>>>> >>>>>>>>>>>> Thanks a lot, >>>>>>>>>>>> -n >>>>>>>>>>>> >>>>>>>>>>>> [0]: http://phoenix.apache.org/phoenix_spark.html >>>>>>>>>>>> [1]: >>>>>>>>>>>> >>>>>>>>>>>> java.lang.IllegalStateException: unread block data >>>>>>>>>>>> at >>>>>>>>>>>> java.io.ObjectInputStream$BlockDataInputStream.setBlockDataMode(ObjectInputStream.java:2424) >>>>>>>>>>>> at >>>>>>>>>>>> java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1383) >>>>>>>>>>>> at >>>>>>>>>>>> java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1993) >>>>>>>>>>>> at >>>>>>>>>>>> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1918) >>>>>>>>>>>> at >>>>>>>>>>>> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801) >>>>>>>>>>>> at >>>>>>>>>>>> java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) >>>>>>>>>>>> at >>>>>>>>>>>> java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) >>>>>>>>>>>> at >>>>>>>>>>>> org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:69) >>>>>>>>>>>> at >>>>>>>>>>>> org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:95) >>>>>>>>>>>> at >>>>>>>>>>>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:194) >>>>>>>>>>>> at >>>>>>>>>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) >>>>>>>>>>>> at >>>>>>>>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) >>>>>>>>>>>> at java.lang.Thread.run(Thread.java:745) >>>>>>>>>>>> >>>>>>>>>>>> >>>>>>>>>>>> On Mon, Dec 21, 2015 at 8:33 AM, James Taylor < >>>>>>>>>>>> [email protected]> wrote: >>>>>>>>>>>> >>>>>>>>>>>>> Thanks for remembering about the docs, Josh. >>>>>>>>>>>>> >>>>>>>>>>>>> On Mon, Dec 21, 2015 at 8:27 AM, Josh Mahonin < >>>>>>>>>>>>> [email protected]> wrote: >>>>>>>>>>>>> >>>>>>>>>>>>>> Just an update for anyone interested, PHOENIX-2503 was just >>>>>>>>>>>>>> committed for 4.7.0 and the docs have been updated to include >>>>>>>>>>>>>> these samples >>>>>>>>>>>>>> for PySpark users. >>>>>>>>>>>>>> >>>>>>>>>>>>>> https://phoenix.apache.org/phoenix_spark.html >>>>>>>>>>>>>> >>>>>>>>>>>>>> Josh >>>>>>>>>>>>>> >>>>>>>>>>>>>> On Thu, Dec 10, 2015 at 1:20 PM, Josh Mahonin < >>>>>>>>>>>>>> [email protected]> wrote: >>>>>>>>>>>>>> >>>>>>>>>>>>>>> Hey Nick, >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> I think this used to work, and will again once PHOENIX-2503 >>>>>>>>>>>>>>> gets resolved. With the Spark DataFrame support, all the >>>>>>>>>>>>>>> necessary glue is >>>>>>>>>>>>>>> there for Phoenix and pyspark to play nice. With that client >>>>>>>>>>>>>>> JAR (or by >>>>>>>>>>>>>>> overriding the com.fasterxml.jackson JARS), you can do >>>>>>>>>>>>>>> something like: >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> df = sqlContext.read \ >>>>>>>>>>>>>>> .format("org.apache.phoenix.spark") \ >>>>>>>>>>>>>>> .option("table", "TABLE1") \ >>>>>>>>>>>>>>> .option("zkUrl", "localhost:63512") \ >>>>>>>>>>>>>>> .load() >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> And >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> df.write \ >>>>>>>>>>>>>>> .format("org.apache.phoenix.spark") \ >>>>>>>>>>>>>>> .mode("overwrite") \ >>>>>>>>>>>>>>> .option("table", "TABLE1") \ >>>>>>>>>>>>>>> .option("zkUrl", "localhost:63512") \ >>>>>>>>>>>>>>> .save() >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Yes, this should be added to the documentation. I hadn't >>>>>>>>>>>>>>> actually tried this till just now. :) >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> On Wed, Dec 9, 2015 at 6:39 PM, Nick Dimiduk < >>>>>>>>>>>>>>> [email protected]> wrote: >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> Heya, >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> Has anyone any experience using phoenix-spark integration >>>>>>>>>>>>>>>> from pyspark instead of scala? Folks prefer python around >>>>>>>>>>>>>>>> here... >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> I did find this example [0] of using HBaseOutputFormat from >>>>>>>>>>>>>>>> pyspark, haven't tried extending it for phoenix. Maybe someone >>>>>>>>>>>>>>>> with more >>>>>>>>>>>>>>>> experience in pyspark knows better? Would be a great addition >>>>>>>>>>>>>>>> to our >>>>>>>>>>>>>>>> documentation. >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> Thanks, >>>>>>>>>>>>>>>> Nick >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> [0]: >>>>>>>>>>>>>>>> https://github.com/apache/spark/blob/master/examples/src/main/python/hbase_outputformat.py >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> >>>>>>>>>>>>>> >>>>>>>>>>>>> >>>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>> >>>>>>>>> >>>>>>>> >>>>>>> >>>>>> >>>>> >>>> >>> >> >
