Re: Sorting within partitions is not maintained in parquet?

2016-08-11 Thread Michael Armbrust
This is an optimization to avoid overloading the scheduler with many small
tasks.  It bin-packs data into tasks based on the file size.

You can disable it by setting spark.sql.files.openCostInBytes very high
(higher than spark.sql.files.maxPartitionBytes).

On Thu, Aug 11, 2016 at 4:27 AM, Hyukjin Kwon  wrote:

> I just took a quick look for this. It seems not parquet-specific problem
> but for datasources implimenting FileFormat.
>
> In 1.6, it seems apparently partitions are made per file but in 2.0
> partition can hold multiple files.
>
> So, in your case files are miltiple but partitions are fewer, meaning each
> partition is not sorted although each part-file is sorted.
>
> It seems this PR https://github.com/apache/spark/pull/12095 is related.
>
> I could not test before/after this PR because I don't have the access to
> my computer for now (it's my phone) but I am sure the PR is related.
>
> Maybe we need an option to enable/disable this?
>
> I appreciate if any gives some feedback.
>
> Thanks!
>
> On 11 Aug 2016 3:23 p.m., "Jason Moore" 
> wrote:
>
>> Hi,
>>
>>
>>
>> It seems that something changed between Spark 1.6.2 and 2.0.0 that I
>> wasn’t expecting.
>>
>>
>>
>> If I have a DataFrame with records sorted within each partition, and I
>> write it to parquet and read back from the parquet, previously the records
>> would be iterated through in the same order they were written (assuming no
>> shuffle has taken place).  But this doesn’t seem to be the case anymore.
>> Below is the code to reproduce in a spark-shell.
>>
>>
>>
>> Was this change expected?
>>
>>
>>
>> Thanks,
>>
>> Jason.
>>
>>
>>
>>
>>
>> import org.apache.spark.sql._
>>
>> def isSorted[T](self: DataFrame, mapping: Row => T)(implicit ordering:
>> Ordering[T]) = {
>>
>>   import self.sqlContext.implicits._
>>
>>   import ordering._
>>
>>   self
>>
>> .mapPartitions(rows => {
>>
>>   val isSorted = rows
>>
>> .map(mapping)
>>
>> .sliding(2) // all adjacent pairs
>>
>> .forall {
>>
>>   case x :: y :: Nil => x <= y
>>
>>   case x :: Nil => true
>>
>>   case Nil => true
>>
>> }
>>
>>
>>
>>   Iterator(isSorted)
>>
>> })
>>
>> .reduce(_ && _)
>>
>> }
>>
>>
>>
>> // in Spark 2.0.0
>>
>> spark.range(10).toDF("id").registerTempTable("input")
>>
>> spark.sql("SELECT id FROM input DISTRIBUTE BY id SORT BY
>> id").write.mode("overwrite").parquet("input.parquet")
>>
>> isSorted(spark.read.parquet("input.parquet"), _.getAs[Long]("id"))
>>
>> // FALSE
>>
>>
>>
>> // in Spark 1.6.2
>>
>> sqlContext.range(10).toDF("id").registerTempTable("input")
>>
>> sqlContext.sql("SELECT id FROM input DISTRIBUTE BY id SORT BY
>> id").write.mode("overwrite").parquet("input.parquet")
>>
>> isSorted(sqlContext.read.parquet("input.parquet"), _.getAs[Long]("id"))
>>
>> // TRUE
>>
>>
>>
>


Re: Serving Spark ML models via a regular Python web app

2016-08-11 Thread Michael Allman
Hi Chris,

I was just checking out your project. I mentioned we use MLeap to serve 
predictions from a trained Spark ML RandomForest model. How would I do that 
with pipeline.io ? It isn't clear to me.

Thanks!

Michael

> On Aug 11, 2016, at 9:42 AM, Chris Fregly  wrote:
> 
> And here's a recent slide deck on the pipeline.io  that 
> summarizes what we're working on (all open source):  
> 
> https://www.slideshare.net/mobile/cfregly/advanced-spark-and-tensorflow-meetup-08042016-one-click-spark-ml-pipeline-deploy-to-production
>  
> 
> 
> mleap is heading the wrong direction and reinventing the wheel.  not quite 
> sure where that project will go.  doesn't seem like it will have a long 
> shelf-life in my opinion.
> 
> check out pipeline.io .  some cool stuff in there.
> 
> On Aug 11, 2016, at 9:35 AM, Chris Fregly  > wrote:
> 
>> this is exactly what my http://pipeline.io  project is 
>> addressing.  check it out and send me feedback or create issues at that 
>> github location.
>> 
>> On Aug 11, 2016, at 7:42 AM, Nicholas Chammas > > wrote:
>> 
>>> Thanks Michael for the reference, and thanks Nick for the comprehensive 
>>> overview of existing JIRA discussions about this. I've added myself as a 
>>> watcher on the various tasks.
>>> 
>>> On Thu, Aug 11, 2016 at 3:02 AM Nick Pentreath >> > wrote:
>>> Currently there is no direct way in Spark to serve models without bringing 
>>> in all of Spark as a dependency.
>>> 
>>> For Spark ML, there is actually no way to do it independently of DataFrames 
>>> either (which for single-instance prediction makes things sub-optimal). 
>>> That is covered here: https://issues.apache.org/jira/browse/SPARK-10413 
>>> 
>>> 
>>> So, your options are (in Scala) things like MLeap, PredictionIO, or "roll 
>>> your own". Or you can try to export to some other format such as PMML or 
>>> PFA. Some MLlib models support PMML export, but for ML it is still missing 
>>> (see https://issues.apache.org/jira/browse/SPARK-11171 
>>> ).
>>> 
>>> There is an external project for PMML too (note licensing) - 
>>> https://github.com/jpmml/jpmml-sparkml 
>>>  - which is by now actually quite 
>>> comprehensive. It shows that PMML can represent a pretty large subset of 
>>> typical ML pipeline functionality.
>>> 
>>> On the Python side sadly there is even less - I would say your options are 
>>> pretty much "roll your own" currently, or export in PMML or PFA.
>>> 
>>> Finally, part of the "mllib-local" idea was around enabling this local 
>>> model-serving (for some initial discussion about the future see 
>>> https://issues.apache.org/jira/browse/SPARK-16365 
>>> ).
>>> 
>>> N
>>> 
>>> 
>>> On Thu, 11 Aug 2016 at 06:28 Michael Allman >> > wrote:
>>> Nick,
>>> 
>>> Check out MLeap: https://github.com/TrueCar/mleap 
>>> . It's not python, but we use it in 
>>> production to serve a random forest model trained by a Spark ML pipeline.
>>> 
>>> Thanks,
>>> 
>>> Michael
>>> 
 On Aug 10, 2016, at 7:50 PM, Nicholas Chammas >>> > wrote:
 
 Are there any existing JIRAs covering the possibility of serving up Spark 
 ML models via, for example, a regular Python web app?
 
 The story goes like this: You train your model with Spark on several TB of 
 data, and now you want to use it in a prediction service that you’re 
 building, say with Flask . In principle, you 
 don’t need Spark anymore since you’re just passing individual data points 
 to your model and looking for it to spit some prediction back.
 
 I assume this is something people do today, right? I presume Spark needs 
 to run in their web service to serve up the model. (Sorry, I’m new to the 
 ML side of Spark. 😅)
 
 Are there any JIRAs discussing potential improvements to this story? I did 
 a search, but I’m not sure what exactly to look for. SPARK-4587 
  (model import/export) 
 looks relevant, but doesn’t address the story directly.
 
 Nick
 
>>> 



Re: Serving Spark ML models via a regular Python web app

2016-08-11 Thread Nicholas Chammas
Thanks for the additional reference Chris. Sounds like there are a few
independent projects addressing this story.

On Thu, Aug 11, 2016 at 12:42 PM Chris Fregly  wrote:

> And here's a recent slide deck on the pipeline.io that summarizes what
> we're working on (all open source):
>
>
> https://www.slideshare.net/mobile/cfregly/advanced-spark-and-tensorflow-meetup-08042016-one-click-spark-ml-pipeline-deploy-to-production
>
> mleap is heading the wrong direction and reinventing the wheel.  not quite
> sure where that project will go.  doesn't seem like it will have a long
> shelf-life in my opinion.
>
> check out pipeline.io.  some cool stuff in there.
>
> On Aug 11, 2016, at 9:35 AM, Chris Fregly  wrote:
>
> this is exactly what my http://pipeline.io project is addressing.  check
> it out and send me feedback or create issues at that github location.
>
> On Aug 11, 2016, at 7:42 AM, Nicholas Chammas 
> wrote:
>
> Thanks Michael for the reference, and thanks Nick for the comprehensive
> overview of existing JIRA discussions about this. I've added myself as a
> watcher on the various tasks.
>
> On Thu, Aug 11, 2016 at 3:02 AM Nick Pentreath 
> wrote:
>
>> Currently there is no direct way in Spark to serve models without
>> bringing in all of Spark as a dependency.
>>
>> For Spark ML, there is actually no way to do it independently of
>> DataFrames either (which for single-instance prediction makes things
>> sub-optimal). That is covered here:
>> https://issues.apache.org/jira/browse/SPARK-10413
>>
>> So, your options are (in Scala) things like MLeap, PredictionIO, or "roll
>> your own". Or you can try to export to some other format such as PMML or
>> PFA. Some MLlib models support PMML export, but for ML it is still missing
>> (see https://issues.apache.org/jira/browse/SPARK-11171).
>>
>> There is an external project for PMML too (note licensing) -
>> https://github.com/jpmml/jpmml-sparkml - which is by now actually quite
>> comprehensive. It shows that PMML can represent a pretty large subset of
>> typical ML pipeline functionality.
>>
>> On the Python side sadly there is even less - I would say your options
>> are pretty much "roll your own" currently, or export in PMML or PFA.
>>
>> Finally, part of the "mllib-local" idea was around enabling this local
>> model-serving (for some initial discussion about the future see
>> https://issues.apache.org/jira/browse/SPARK-16365).
>>
>> N
>>
>>
>> On Thu, 11 Aug 2016 at 06:28 Michael Allman  wrote:
>>
>>> Nick,
>>>
>>> Check out MLeap: https://github.com/TrueCar/mleap. It's not python, but
>>> we use it in production to serve a random forest model trained by a Spark
>>> ML pipeline.
>>>
>>> Thanks,
>>>
>>> Michael
>>>
>>> On Aug 10, 2016, at 7:50 PM, Nicholas Chammas <
>>> nicholas.cham...@gmail.com> wrote:
>>>
>>> Are there any existing JIRAs covering the possibility of serving up
>>> Spark ML models via, for example, a regular Python web app?
>>>
>>> The story goes like this: You train your model with Spark on several TB
>>> of data, and now you want to use it in a prediction service that you’re
>>> building, say with Flask . In principle, you
>>> don’t need Spark anymore since you’re just passing individual data points
>>> to your model and looking for it to spit some prediction back.
>>>
>>> I assume this is something people do today, right? I presume Spark needs
>>> to run in their web service to serve up the model. (Sorry, I’m new to the
>>> ML side of Spark. 😅)
>>>
>>> Are there any JIRAs discussing potential improvements to this story? I
>>> did a search, but I’m not sure what exactly to look for. SPARK-4587
>>>  (model
>>> import/export) looks relevant, but doesn’t address the story directly.
>>>
>>> Nick
>>> ​
>>>
>>>
>>>


Re: Serving Spark ML models via a regular Python web app

2016-08-11 Thread Chris Fregly
And here's a recent slide deck on the pipeline.io that summarizes what we're 
working on (all open source):  

https://www.slideshare.net/mobile/cfregly/advanced-spark-and-tensorflow-meetup-08042016-one-click-spark-ml-pipeline-deploy-to-production

mleap is heading the wrong direction and reinventing the wheel.  not quite sure 
where that project will go.  doesn't seem like it will have a long shelf-life 
in my opinion.

check out pipeline.io.  some cool stuff in there.

> On Aug 11, 2016, at 9:35 AM, Chris Fregly  wrote:
> 
> this is exactly what my http://pipeline.io project is addressing.  check it 
> out and send me feedback or create issues at that github location.
> 
>> On Aug 11, 2016, at 7:42 AM, Nicholas Chammas  
>> wrote:
>> 
>> Thanks Michael for the reference, and thanks Nick for the comprehensive 
>> overview of existing JIRA discussions about this. I've added myself as a 
>> watcher on the various tasks.
>> 
>>> On Thu, Aug 11, 2016 at 3:02 AM Nick Pentreath  
>>> wrote:
>>> Currently there is no direct way in Spark to serve models without bringing 
>>> in all of Spark as a dependency.
>>> 
>>> For Spark ML, there is actually no way to do it independently of DataFrames 
>>> either (which for single-instance prediction makes things sub-optimal). 
>>> That is covered here: https://issues.apache.org/jira/browse/SPARK-10413
>>> 
>>> So, your options are (in Scala) things like MLeap, PredictionIO, or "roll 
>>> your own". Or you can try to export to some other format such as PMML or 
>>> PFA. Some MLlib models support PMML export, but for ML it is still missing 
>>> (see https://issues.apache.org/jira/browse/SPARK-11171).
>>> 
>>> There is an external project for PMML too (note licensing) - 
>>> https://github.com/jpmml/jpmml-sparkml - which is by now actually quite 
>>> comprehensive. It shows that PMML can represent a pretty large subset of 
>>> typical ML pipeline functionality.
>>> 
>>> On the Python side sadly there is even less - I would say your options are 
>>> pretty much "roll your own" currently, or export in PMML or PFA.
>>> 
>>> Finally, part of the "mllib-local" idea was around enabling this local 
>>> model-serving (for some initial discussion about the future see 
>>> https://issues.apache.org/jira/browse/SPARK-16365).
>>> 
>>> N
>>> 
>>> 
 On Thu, 11 Aug 2016 at 06:28 Michael Allman  wrote:
 Nick,
 
 Check out MLeap: https://github.com/TrueCar/mleap. It's not python, but we 
 use it in production to serve a random forest model trained by a Spark ML 
 pipeline.
 
 Thanks,
 
 Michael
 
> On Aug 10, 2016, at 7:50 PM, Nicholas Chammas 
>  wrote:
> 
> Are there any existing JIRAs covering the possibility of serving up Spark 
> ML models via, for example, a regular Python web app?
> 
> The story goes like this: You train your model with Spark on several TB 
> of data, and now you want to use it in a prediction service that you’re 
> building, say with Flask. In principle, you don’t need Spark anymore 
> since you’re just passing individual data points to your model and 
> looking for it to spit some prediction back.
> 
> I assume this is something people do today, right? I presume Spark needs 
> to run in their web service to serve up the model. (Sorry, I’m new to the 
> ML side of Spark. 😅)
> 
> Are there any JIRAs discussing potential improvements to this story? I 
> did a search, but I’m not sure what exactly to look for. SPARK-4587 
> (model import/export) looks relevant, but doesn’t address the story 
> directly.
> 
> Nick


Re: Serving Spark ML models via a regular Python web app

2016-08-11 Thread Chris Fregly
this is exactly what my http://pipeline.io project is addressing.  check it out 
and send me feedback or create issues at that github location.

> On Aug 11, 2016, at 7:42 AM, Nicholas Chammas  
> wrote:
> 
> Thanks Michael for the reference, and thanks Nick for the comprehensive 
> overview of existing JIRA discussions about this. I've added myself as a 
> watcher on the various tasks.
> 
>> On Thu, Aug 11, 2016 at 3:02 AM Nick Pentreath  
>> wrote:
>> Currently there is no direct way in Spark to serve models without bringing 
>> in all of Spark as a dependency.
>> 
>> For Spark ML, there is actually no way to do it independently of DataFrames 
>> either (which for single-instance prediction makes things sub-optimal). That 
>> is covered here: https://issues.apache.org/jira/browse/SPARK-10413
>> 
>> So, your options are (in Scala) things like MLeap, PredictionIO, or "roll 
>> your own". Or you can try to export to some other format such as PMML or 
>> PFA. Some MLlib models support PMML export, but for ML it is still missing 
>> (see https://issues.apache.org/jira/browse/SPARK-11171).
>> 
>> There is an external project for PMML too (note licensing) - 
>> https://github.com/jpmml/jpmml-sparkml - which is by now actually quite 
>> comprehensive. It shows that PMML can represent a pretty large subset of 
>> typical ML pipeline functionality.
>> 
>> On the Python side sadly there is even less - I would say your options are 
>> pretty much "roll your own" currently, or export in PMML or PFA.
>> 
>> Finally, part of the "mllib-local" idea was around enabling this local 
>> model-serving (for some initial discussion about the future see 
>> https://issues.apache.org/jira/browse/SPARK-16365).
>> 
>> N
>> 
>> 
>>> On Thu, 11 Aug 2016 at 06:28 Michael Allman  wrote:
>>> Nick,
>>> 
>>> Check out MLeap: https://github.com/TrueCar/mleap. It's not python, but we 
>>> use it in production to serve a random forest model trained by a Spark ML 
>>> pipeline.
>>> 
>>> Thanks,
>>> 
>>> Michael
>>> 
 On Aug 10, 2016, at 7:50 PM, Nicholas Chammas  
 wrote:
 
 Are there any existing JIRAs covering the possibility of serving up Spark 
 ML models via, for example, a regular Python web app?
 
 The story goes like this: You train your model with Spark on several TB of 
 data, and now you want to use it in a prediction service that you’re 
 building, say with Flask. In principle, you don’t need Spark anymore since 
 you’re just passing individual data points to your model and looking for 
 it to spit some prediction back.
 
 I assume this is something people do today, right? I presume Spark needs 
 to run in their web service to serve up the model. (Sorry, I’m new to the 
 ML side of Spark. 😅)
 
 Are there any JIRAs discussing potential improvements to this story? I did 
 a search, but I’m not sure what exactly to look for. SPARK-4587 (model 
 import/export) looks relevant, but doesn’t address the story directly.
 
 Nick


Re: Serving Spark ML models via a regular Python web app

2016-08-11 Thread Nicholas Chammas
Thanks Michael for the reference, and thanks Nick for the comprehensive
overview of existing JIRA discussions about this. I've added myself as a
watcher on the various tasks.

On Thu, Aug 11, 2016 at 3:02 AM Nick Pentreath 
wrote:

> Currently there is no direct way in Spark to serve models without bringing
> in all of Spark as a dependency.
>
> For Spark ML, there is actually no way to do it independently of
> DataFrames either (which for single-instance prediction makes things
> sub-optimal). That is covered here:
> https://issues.apache.org/jira/browse/SPARK-10413
>
> So, your options are (in Scala) things like MLeap, PredictionIO, or "roll
> your own". Or you can try to export to some other format such as PMML or
> PFA. Some MLlib models support PMML export, but for ML it is still missing
> (see https://issues.apache.org/jira/browse/SPARK-11171).
>
> There is an external project for PMML too (note licensing) -
> https://github.com/jpmml/jpmml-sparkml - which is by now actually quite
> comprehensive. It shows that PMML can represent a pretty large subset of
> typical ML pipeline functionality.
>
> On the Python side sadly there is even less - I would say your options are
> pretty much "roll your own" currently, or export in PMML or PFA.
>
> Finally, part of the "mllib-local" idea was around enabling this local
> model-serving (for some initial discussion about the future see
> https://issues.apache.org/jira/browse/SPARK-16365).
>
> N
>
>
> On Thu, 11 Aug 2016 at 06:28 Michael Allman  wrote:
>
>> Nick,
>>
>> Check out MLeap: https://github.com/TrueCar/mleap. It's not python, but
>> we use it in production to serve a random forest model trained by a Spark
>> ML pipeline.
>>
>> Thanks,
>>
>> Michael
>>
>> On Aug 10, 2016, at 7:50 PM, Nicholas Chammas 
>> wrote:
>>
>> Are there any existing JIRAs covering the possibility of serving up Spark
>> ML models via, for example, a regular Python web app?
>>
>> The story goes like this: You train your model with Spark on several TB
>> of data, and now you want to use it in a prediction service that you’re
>> building, say with Flask . In principle, you
>> don’t need Spark anymore since you’re just passing individual data points
>> to your model and looking for it to spit some prediction back.
>>
>> I assume this is something people do today, right? I presume Spark needs
>> to run in their web service to serve up the model. (Sorry, I’m new to the
>> ML side of Spark. 😅)
>>
>> Are there any JIRAs discussing potential improvements to this story? I
>> did a search, but I’m not sure what exactly to look for. SPARK-4587
>>  (model import/export)
>> looks relevant, but doesn’t address the story directly.
>>
>> Nick
>> ​
>>
>>
>>


Re: Sorting within partitions is not maintained in parquet?

2016-08-11 Thread Hyukjin Kwon
I just took a quick look for this. It seems not parquet-specific problem
but for datasources implimenting FileFormat.

In 1.6, it seems apparently partitions are made per file but in 2.0
partition can hold multiple files.

So, in your case files are miltiple but partitions are fewer, meaning each
partition is not sorted although each part-file is sorted.

It seems this PR https://github.com/apache/spark/pull/12095 is related.

I could not test before/after this PR because I don't have the access to my
computer for now (it's my phone) but I am sure the PR is related.

Maybe we need an option to enable/disable this?

I appreciate if any gives some feedback.

Thanks!

On 11 Aug 2016 3:23 p.m., "Jason Moore"  wrote:

> Hi,
>
>
>
> It seems that something changed between Spark 1.6.2 and 2.0.0 that I
> wasn’t expecting.
>
>
>
> If I have a DataFrame with records sorted within each partition, and I
> write it to parquet and read back from the parquet, previously the records
> would be iterated through in the same order they were written (assuming no
> shuffle has taken place).  But this doesn’t seem to be the case anymore.
> Below is the code to reproduce in a spark-shell.
>
>
>
> Was this change expected?
>
>
>
> Thanks,
>
> Jason.
>
>
>
>
>
> import org.apache.spark.sql._
>
> def isSorted[T](self: DataFrame, mapping: Row => T)(implicit ordering:
> Ordering[T]) = {
>
>   import self.sqlContext.implicits._
>
>   import ordering._
>
>   self
>
> .mapPartitions(rows => {
>
>   val isSorted = rows
>
> .map(mapping)
>
> .sliding(2) // all adjacent pairs
>
> .forall {
>
>   case x :: y :: Nil => x <= y
>
>   case x :: Nil => true
>
>   case Nil => true
>
> }
>
>
>
>   Iterator(isSorted)
>
> })
>
> .reduce(_ && _)
>
> }
>
>
>
> // in Spark 2.0.0
>
> spark.range(10).toDF("id").registerTempTable("input")
>
> spark.sql("SELECT id FROM input DISTRIBUTE BY id SORT BY
> id").write.mode("overwrite").parquet("input.parquet")
>
> isSorted(spark.read.parquet("input.parquet"), _.getAs[Long]("id"))
>
> // FALSE
>
>
>
> // in Spark 1.6.2
>
> sqlContext.range(10).toDF("id").registerTempTable("input")
>
> sqlContext.sql("SELECT id FROM input DISTRIBUTE BY id SORT BY
> id").write.mode("overwrite").parquet("input.parquet")
>
> isSorted(sqlContext.read.parquet("input.parquet"), _.getAs[Long]("id"))
>
> // TRUE
>
>
>


Who controls 'databricks-jenkins'?

2016-08-11 Thread Sean Owen
Not a big deal but 'he' is commenting on a lot of ancient PRs for some
reason, like https://github.com/apache/spark/pull/51 and it generates
mails to the list. I assume this is a misconfiguration somewhere.

-
To unsubscribe e-mail: dev-unsubscr...@spark.apache.org



Re: Serving Spark ML models via a regular Python web app

2016-08-11 Thread Nick Pentreath
Currently there is no direct way in Spark to serve models without bringing
in all of Spark as a dependency.

For Spark ML, there is actually no way to do it independently of DataFrames
either (which for single-instance prediction makes things sub-optimal).
That is covered here: https://issues.apache.org/jira/browse/SPARK-10413

So, your options are (in Scala) things like MLeap, PredictionIO, or "roll
your own". Or you can try to export to some other format such as PMML or
PFA. Some MLlib models support PMML export, but for ML it is still missing
(see https://issues.apache.org/jira/browse/SPARK-11171).

There is an external project for PMML too (note licensing) -
https://github.com/jpmml/jpmml-sparkml - which is by now actually quite
comprehensive. It shows that PMML can represent a pretty large subset of
typical ML pipeline functionality.

On the Python side sadly there is even less - I would say your options are
pretty much "roll your own" currently, or export in PMML or PFA.

Finally, part of the "mllib-local" idea was around enabling this local
model-serving (for some initial discussion about the future see
https://issues.apache.org/jira/browse/SPARK-16365).

N

On Thu, 11 Aug 2016 at 06:28 Michael Allman  wrote:

> Nick,
>
> Check out MLeap: https://github.com/TrueCar/mleap. It's not python, but
> we use it in production to serve a random forest model trained by a Spark
> ML pipeline.
>
> Thanks,
>
> Michael
>
> On Aug 10, 2016, at 7:50 PM, Nicholas Chammas 
> wrote:
>
> Are there any existing JIRAs covering the possibility of serving up Spark
> ML models via, for example, a regular Python web app?
>
> The story goes like this: You train your model with Spark on several TB of
> data, and now you want to use it in a prediction service that you’re
> building, say with Flask . In principle, you
> don’t need Spark anymore since you’re just passing individual data points
> to your model and looking for it to spit some prediction back.
>
> I assume this is something people do today, right? I presume Spark needs
> to run in their web service to serve up the model. (Sorry, I’m new to the
> ML side of Spark. 😅)
>
> Are there any JIRAs discussing potential improvements to this story? I did
> a search, but I’m not sure what exactly to look for. SPARK-4587
>  (model import/export)
> looks relevant, but doesn’t address the story directly.
>
> Nick
> ​
>
>
>