Re: encoders for more complex types

2016-10-27 Thread Michael Armbrust
I would categorize these as bugs. We should (but probably don't fully yet) support arbitrary nesting as long as you use basic collections / case classes / primitives. Please do open JIRAs as you find problems. On Thu, Oct 27, 2016 at 1:05 PM, Koert Kuipers wrote: > well i

Re: [VOTE] Release Apache Spark 2.0.2 (RC1)

2016-10-27 Thread Michael Armbrust
+1 On Oct 27, 2016 12:19 AM, "Reynold Xin" wrote: > Greetings from Spark Summit Europe at Brussels. > > Please vote on releasing the following candidate as Apache Spark version > 2.0.2. The vote is open until Sun, Oct 30, 2016 at 00:30 PDT and passes if > a majority of at

[jira] [Resolved] (SPARK-17770) Make ObjectType SQL Type Public

2016-10-26 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17770?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust resolved SPARK-17770. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request 15453

Re: Dataframe schema...

2016-10-26 Thread Michael Armbrust
On Fri, Oct 21, 2016 at 8:40 PM, Koert Kuipers wrote: > This rather innocent looking optimization flag nullable has caused a lot > of bugs... Makes me wonder if we are better off without it > Yes... my most regretted design decision :( Please give thoughts here:

Re: Watermarking in Structured Streaming to drop late data

2016-10-26 Thread Michael Armbrust
And the JIRA: https://issues.apache.org/jira/browse/SPARK-18124 On Wed, Oct 26, 2016 at 4:56 PM, Tathagata Das wrote: > Hey all, > > We are planning implement watermarking in Structured Streaming that would > allow us handle late, out-of-order data better. Specially, when

[jira] [Commented] (SPARK-18084) write.partitionBy() does not recognize nested columns that select() can access

2016-10-26 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-18084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15610097#comment-15610097 ] Michael Armbrust commented on SPARK-18084: -- I'd put this in the general class of issues where

[jira] [Updated] (SPARK-18127) Add hooks and extension points to Spark

2016-10-26 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-18127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust updated SPARK-18127: - Assignee: Herman van Hovell > Add hooks and extension points to Sp

Re: [Spark 2.0.1] Error in generated code, possible regression?

2016-10-26 Thread Michael Armbrust
uidelines for > tracking down the context of this generated code? > > On Wed, Oct 26, 2016 at 3:42 PM Michael Armbrust <mich...@databricks.com> > wrote: > >> If you have a reproduction you can post for this, it would be great if >> you could open a JIRA. >> >&

Re: [Spark 2.0.1] Error in generated code, possible regression?

2016-10-26 Thread Michael Armbrust
If you have a reproduction you can post for this, it would be great if you could open a JIRA. On Mon, Oct 24, 2016 at 6:21 PM, Efe Selcuk wrote: > I have an application that works in 2.0.0 but has been dying at runtime on > the 2.0.1 distribution. > > at

Re: Resiliency with SparkStreaming - fileStream

2016-10-26 Thread Michael Armbrust
I'll answer in the context of structured streaming (the new streaming API build on DataFrames). When reading from files, the FileSource, records which files are included in each batch inside of the given checkpointLocation. If you fail in the middle of a batch, the streaming engine will retry

Re: getting encoder implicits to be more accurate

2016-10-26 Thread Michael Armbrust
e great > > On Wed, Oct 26, 2016 at 5:10 PM, Koert Kuipers <ko...@tresata.com> wrote: > >> i use kryo for the whole thing currently >> >> it would be better to use it for the subtree >> >> On Wed, Oct 26, 2016 at 5:06 PM, Michael Armbrust <mich...@da

[jira] [Created] (SPARK-18122) Fallback to Kryo for unknown classes in ExpressionEncoder

2016-10-26 Thread Michael Armbrust (JIRA)
Michael Armbrust created SPARK-18122: Summary: Fallback to Kryo for unknown classes in ExpressionEncoder Key: SPARK-18122 URL: https://issues.apache.org/jira/browse/SPARK-18122 Project: Spark

Re: getting encoder implicits to be more accurate

2016-10-26 Thread Michael Armbrust
> >> for us the result is that say an implicit for Seq[(Int, Seq[(String, >> Int)])] will create a new ExpressionEncoder(), while an implicit for >> Seq[(Int, Set[(String, Int)])] produces a Encoders.kryoEncoder() >> >> On Wed, Oct 26, 2016 at 3:50 PM, Michael Armbrust &l

Re: getting encoder implicits to be more accurate

2016-10-26 Thread Michael Armbrust
s there isn't an encoder for, right? >> >> On Wed, Oct 26, 2016 at 11:18 AM, Michael Armbrust < >> mich...@databricks.com> wrote: >> >>> Hmm, that is unfortunate. Maybe the best solution is to add support for >>> sets? I don't think that would be su

Re: Straw poll: dropping support for things like Scala 2.10

2016-10-26 Thread Michael Armbrust
+1 On Wed, Oct 26, 2016 at 11:26 AM, Reynold Xin wrote: > We can do the following concrete proposal: > > 1. Plan to remove support for Java 7 / Scala 2.10 in Spark 2.2.0 (Mar/Apr > 2017). > > 2. In Spark 2.1.0 release, aggressively and explicitly announce the > deprecation

Re: getting encoder implicits to be more accurate

2016-10-26 Thread Michael Armbrust
Hmm, that is unfortunate. Maybe the best solution is to add support for sets? I don't think that would be super hard. On Tue, Oct 25, 2016 at 8:52 PM, Koert Kuipers wrote: > i am trying to use encoders as a typeclass where if it fails to find an > ExpressionEncoder it falls

Re: LIMIT issue of SparkSQL

2016-10-24 Thread Michael Armbrust
> *Disclaimer:* Use it at your own risk. Any and all responsibility for any > loss, damage or destruction of data or any other property which may arise > from relying on this email's technical content is explicitly disclaimed. > The author will in no case be liable for any monetary damages

Re: LIMIT issue of SparkSQL

2016-10-23 Thread Michael Armbrust
- dev + user Can you give more info about the query? Maybe a full explain()? Are you using a datasource like JDBC? The API does not currently push down limits, but the documentation talks about how you can use a query instead of a table if that is what you are looking to do. On Mon, Oct 24,

Re: LIMIT issue of SparkSQL

2016-10-23 Thread Michael Armbrust
- dev + user Can you give more info about the query? Maybe a full explain()? Are you using a datasource like JDBC? The API does not currently push down limits, but the documentation talks about how you can use a query instead of a table if that is what you are looking to do. On Mon, Oct 24,

Re: Ran in to a bug in Broadcast Hash Join

2016-10-22 Thread Michael Armbrust
2.0.0 or 2.0.1? There are several correctness fixes in the latter. On Oct 22, 2016 2:14 PM, "Jeremy Davis" wrote: > > Hello, I ran in to a bug with Broadcast Hash Join in Spark 2.0. (Running > on EMR) > If I just toggle spark.sql.autoBroadcastJoinThreshold=-1 then the

[jira] [Commented] (SPARK-17829) Stable format for offset log

2016-10-21 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15595716#comment-15595716 ] Michael Armbrust commented on SPARK-17829: -- Yeah, I agree. I think it could be really simple

[jira] [Updated] (SPARK-17829) Stable format for offset log

2016-10-20 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17829?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust updated SPARK-17829: - Assignee: Tyson Condie > Stable format for offset

Re: StructuredStreaming status

2016-10-20 Thread Michael Armbrust
> > On a personal note, I'm quite surprised that this is all the progress in > Structured Streaming over the last three months since 2.0 was released. I > was under the impression that this was one of the biggest things that the > Spark community actively works on, but that is clearly not the

Re: Dataframe schema...

2016-10-20 Thread Michael Armbrust
ains the mixed containsNull = true/false. > Let me know if this helps. > > Thanks, > Muthu > > > > On Wed, Oct 19, 2016 at 6:21 PM, Michael Armbrust <mich...@databricks.com> > wrote: > >> Nullable is just a hint to the optimizer that its impossible for there t

Re: StructuredStreaming status

2016-10-20 Thread Michael Armbrust
> > let’s say we would have implemented distinct count by saving a map with > the key being the distinct value and the value being the last time we saw > this value. This would mean that we wouldn’t really need to save all the > steps in the middle and copy the data, we could only save the last

Re: How does Spark determine in-memory partition count when reading Parquet ~files?

2016-10-19 Thread Michael Armbrust
In spark 2.0 we bin-pack small files into a single task to avoid overloading the scheduler. If you want a specific number of partitions you should repartition. If you want to disable this optimization you can set the file open cost very high: spark.sql.files.openCostInBytes On Tue, Oct 18, 2016

Re: Dataframe schema...

2016-10-19 Thread Michael Armbrust
Nullable is just a hint to the optimizer that its impossible for there to be a null value in this column, so that it can avoid generating code for null-checks. When in doubt, we set nullable=true since it is always safer to check. Why in particular are you trying to change the nullability of the

Re: Why the json file used by sparkSession.read.json must be a valid json object per line

2016-10-19 Thread Michael Armbrust
On Sun, Oct 16, 2016 at 3:50 AM, wrote: > Think of it as jsonl instead of a json file. > Point people at this if they need an official looking spec: > http://jsonlines.org/ > That link is awesome. I think it would be great if someone could open a PR to add this to our

Re: StructuredStreaming status

2016-10-19 Thread Michael Armbrust
On Wed, Oct 19, 2016 at 2:45 PM, Michael Armbrust > <mich...@databricks.com> wrote: > > Anything that is actively being designed should be in JIRA, and it seems > > like you found most of it. In general, release windows can be found on > the > > wiki. > > > &

Re: StructuredStreaming status

2016-10-19 Thread Michael Armbrust
Anything that is actively being designed should be in JIRA, and it seems like you found most of it. In general, release windows can be found on the wiki . 2.1 has a lot of stability fixes as well as the kafka support you mentioned.

[jira] [Resolved] (SPARK-17900) Mark the following Spark SQL APIs as stable

2016-10-14 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17900?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust resolved SPARK-17900. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request 15469

[jira] [Commented] (SPARK-17812) More granular control of starting offsets (assign)

2016-10-14 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15576840#comment-15576840 ] Michael Armbrust commented on SPARK-17812: -- That sounds pretty good to me, with one question

[jira] [Commented] (SPARK-17813) Maximum data per trigger

2016-10-14 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17813?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15576826#comment-15576826 ] Michael Armbrust commented on SPARK-17813: -- I think its okay to ignore compacted topics

[jira] [Resolved] (SPARK-11775) Allow PySpark to register Java UDF

2016-10-14 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-11775?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust resolved SPARK-11775. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request 9766

[jira] [Commented] (SPARK-12776) Implement Python API for Datasets

2016-10-14 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-12776?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15576739#comment-15576739 ] Michael Armbrust commented on SPARK-12776: -- I would love to see better support here, but I don't

[jira] [Resolved] (SPARK-16063) Add storageLevel to Dataset

2016-10-14 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-16063?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust resolved SPARK-16063. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request 13780

Re: DataFrameReader Schema Supersedes Schema Provided by Encoder, Renders Fields Nullable

2016-10-14 Thread Michael Armbrust
> > Additionally, shall I go ahead and open a ticket pointing out the missing > call to .asNullable in the streaming reader? > Yes please! This probably affects correctness.

[jira] [Assigned] (SPARK-17812) More granular control of starting offsets (assign)

2016-10-13 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust reassigned SPARK-17812: Assignee: Cody Koeninger > More granular control of starting offsets (ass

[jira] [Commented] (SPARK-17812) More granular control of starting offsets (assign)

2016-10-13 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15573786#comment-15573786 ] Michael Armbrust commented on SPARK-17812: -- Please do work on it. It might be good to update

[jira] [Resolved] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-10-13 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17368?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust resolved SPARK-17368. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request 15284

[jira] [Commented] (SPARK-17812) More granular control of starting offsets (assign)

2016-10-13 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15573677#comment-15573677 ] Michael Armbrust commented on SPARK-17812: -- bq. with your proposed interface, what, as a user

[jira] [Commented] (SPARK-17812) More granular control of starting offsets (assign)

2016-10-13 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15573526#comment-15573526 ] Michael Armbrust commented on SPARK-17812: -- As far as I understand it, {{auto.offset.reset

[jira] [Comment Edited] (SPARK-17812) More granular control of starting offsets (assign)

2016-10-13 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15573459#comment-15573459 ] Michael Armbrust edited comment on SPARK-17812 at 10/13/16 10:53 PM

[jira] [Comment Edited] (SPARK-17812) More granular control of starting offsets (assign)

2016-10-13 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15573459#comment-15573459 ] Michael Armbrust edited comment on SPARK-17812 at 10/13/16 10:53 PM

[jira] [Commented] (SPARK-17812) More granular control of starting offsets (assign)

2016-10-13 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15573459#comment-15573459 ] Michael Armbrust commented on SPARK-17812: -- +1 to the suggested was of subscribing

Re: DataFrameReader Schema Supersedes Schema Provided by Encoder, Renders Fields Nullable

2016-10-13 Thread Michael Armbrust
There is a lot of confusion around nullable

[jira] [Commented] (SPARK-17344) Kafka 0.8 support for Structured Streaming

2016-10-11 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15567321#comment-15567321 ] Michael Armbrust commented on SPARK-17344: -- These are good questions. A few thoughts: bq. How

[jira] [Updated] (SPARK-17812) More granular control of starting offsets (assign)

2016-10-11 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust updated SPARK-17812: - Summary: More granular control of starting offsets (assign) (was: More granular control

[jira] [Commented] (SPARK-17812) More granular control of starting offsets

2016-10-11 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15566392#comment-15566392 ] Michael Armbrust commented on SPARK-17812: -- For the seeking back {{X}} offsets use case, I

[jira] [Updated] (SPARK-17812) More granular control of starting offsets

2016-10-11 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust updated SPARK-17812: - Description: Right now you can only run a Streaming Query starting from either

[jira] [Commented] (SPARK-17344) Kafka 0.8 support for Structured Streaming

2016-10-11 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15566191#comment-15566191 ] Michael Armbrust commented on SPARK-17344: -- I think the fact that CDH is still distributing 0.9

Re: StructuredStreaming Custom Sinks (motivated by Structured Streaming Machine Learning)

2016-10-11 Thread Michael Armbrust
This is super helpful, thanks for writing it up! > *Delivering low latency, high throughput, and stability simultaneously:* Right > now, our own tests indicate you can get at most two of these > characteristics out of Spark Streaming at the same time. I know of two > parties that have abandoned

[jira] [Resolved] (SPARK-17830) Annotate Spark SQL public APIs with InterfaceStability

2016-10-10 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17830?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust resolved SPARK-17830. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request 15392

[jira] [Commented] (SPARK-8957) Backport Hive 1.X support to Branch 1.4

2016-10-07 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-8957?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15557139#comment-15557139 ] Michael Armbrust commented on SPARK-8957: - Yeah, close it. > Backport Hive 1.X support to Bra

Re: Kafaka 0.8, 0.9 in Structured Streaming

2016-10-07 Thread Michael Armbrust
> > Without a hell of a lot more work, Assign would be the only strategy > usable. How would the current "subscribe" break?

[jira] [Reopened] (SPARK-17344) Kafka 0.8 support for Structured Streaming

2016-10-07 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17344?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust reopened SPARK-17344: -- I'm going to reopen for now since there is interests and you can't vote on closed issues

Re: Kafaka 0.8, 0.9 in Structured Streaming

2016-10-07 Thread Michael Armbrust
> 0.10 consumers won't work on an earlier broker. > Earlier consumers will (should?) work on a 0.10 broker. > This lines up with my testing. Is there a page I'm missing that describes this? Like does a 0.9 client work with 0.8 broker? Is it always old clients can talk to new brokers but not

Re: Kafaka 0.8, 0.9 in Structured Streaming

2016-10-07 Thread Michael Armbrust
> > The implementation is totally and completely different however, in ways > that leak to the end user. Can you elaborate? Especially in the context of the interface provided by structured streaming.

Kafaka 0.8, 0.9 in Structured Streaming

2016-10-07 Thread Michael Armbrust
We recently merged support for Kafak 0.10.0 in Structured Streaming, but I've been hearing a few people tell me that they are stuck on an older version of Kafka and cannot upgrade. I'm considering revisiting SPARK-17344 , but it would be good to

[jira] [Created] (SPARK-17829) Stable format for offset log

2016-10-07 Thread Michael Armbrust (JIRA)
Michael Armbrust created SPARK-17829: Summary: Stable format for offset log Key: SPARK-17829 URL: https://issues.apache.org/jira/browse/SPARK-17829 Project: Spark Issue Type: Sub-task

[jira] [Commented] (SPARK-15406) Structured streaming support for consuming from Kafka

2016-10-06 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15554150#comment-15554150 ] Michael Armbrust commented on SPARK-15406: -- The SQL offsets are stored in a WAL

[jira] [Commented] (SPARK-15406) Structured streaming support for consuming from Kafka

2016-10-06 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15554027#comment-15554027 ] Michael Armbrust commented on SPARK-15406: -- It would be helpful if you could enumerate problems

[jira] [Commented] (SPARK-17815) Report committed offsets

2016-10-06 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17815?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15553900#comment-15553900 ] Michael Armbrust commented on SPARK-17815: -- The only reason its blocked is because setting

[jira] [Commented] (SPARK-15406) Structured streaming support for consuming from Kafka

2016-10-06 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15553852#comment-15553852 ] Michael Armbrust commented on SPARK-15406: -- Restarting a query is just running that query again

[jira] [Created] (SPARK-17815) Report committed offsets

2016-10-06 Thread Michael Armbrust (JIRA)
Michael Armbrust created SPARK-17815: Summary: Report committed offsets Key: SPARK-17815 URL: https://issues.apache.org/jira/browse/SPARK-17815 Project: Spark Issue Type: Sub-task

[jira] [Commented] (SPARK-15406) Structured streaming support for consuming from Kafka

2016-10-06 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15553405#comment-15553405 ] Michael Armbrust commented on SPARK-15406: -- The programming guide has a pretty good description

[jira] [Commented] (SPARK-15406) Structured streaming support for consuming from Kafka

2016-10-06 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15553328#comment-15553328 ] Michael Armbrust commented on SPARK-15406: -- This missed 2.0.1, but I'm hopeful

[jira] [Commented] (SPARK-15406) Structured streaming support for consuming from Kafka

2016-10-06 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15553212#comment-15553212 ] Michael Armbrust commented on SPARK-15406: -- We are definitly not done! I added a few extra

[jira] [Created] (SPARK-17813) Maximum data per trigger

2016-10-06 Thread Michael Armbrust (JIRA)
Michael Armbrust created SPARK-17813: Summary: Maximum data per trigger Key: SPARK-17813 URL: https://issues.apache.org/jira/browse/SPARK-17813 Project: Spark Issue Type: Sub-task

[jira] [Created] (SPARK-17812) More granular control of starting offsets

2016-10-06 Thread Michael Armbrust (JIRA)
Michael Armbrust created SPARK-17812: Summary: More granular control of starting offsets Key: SPARK-17812 URL: https://issues.apache.org/jira/browse/SPARK-17812 Project: Spark Issue Type

[jira] [Resolved] (SPARK-17780) Report NoClassDefFoundError in StreamExecution

2016-10-06 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17780?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust resolved SPARK-17780. -- Resolution: Fixed Fix Version/s: 2.1.0 2.0.2 Issue resolved

Re: StructuredStreaming Custom Sinks (motivated by Structured Streaming Machine Learning)

2016-10-06 Thread Michael Armbrust
to get some idea of the current plans and progress? I get asked a lot about > when Structured Streaming will be a viable replacement for Spark Streaming, > and I like to be able to give accurate advice. > > Fred > > On Tue, Oct 4, 2016 at 3:02 PM, Michael Armbrust <mic

[jira] [Commented] (SPARK-17344) Kafka 0.8 support for Structured Streaming

2016-10-06 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15552915#comment-15552915 ] Michael Armbrust commented on SPARK-17344: -- BTW, people should still comment here

Re: StructuredStreaming Custom Sinks (motivated by Structured Streaming Machine Learning)

2016-10-04 Thread Michael Armbrust
> > I don't quite understand why exposing it indirectly through a typed > interface should be delayed before finalizing the API. > Spark has a long history of maintaining binary compatibility in its public APIs. I strongly believe this is

[jira] [Commented] (SPARK-17764) to_json function for parsing Structs to json Strings

2016-10-03 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17764?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15543229#comment-15543229 ] Michael Armbrust commented on SPARK-17764: -- I would start by looking at the implementation

[jira] [Updated] (SPARK-17764) to_json function for parsing Structs to json Strings

2016-10-03 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17764?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust updated SPARK-17764: - Target Version/s: 2.1.0 > to_json function for parsing Structs to json Stri

Re: Catalyst - ObjectType for Encoders

2016-09-30 Thread Michael Armbrust
I'd be okay removing that modifier, with one caveat. The code in org.apache.spark.sql.catalyst.* is purposefully excluded from published documentation and does not have the same compatibility guarantees as the rest of the Spark's Public APIs. We leave most of it not "private" so that advanced

Re: Questions about DataFrame's filter()

2016-09-29 Thread Michael Armbrust
-dev +user It surprises me as `filter()` takes a Column, not a `Row => Boolean`. There are several overloaded versions of Dataset.filter(...) def filter(func: FilterFunction[T]): Dataset[T] def filter(func: (T) ⇒ Boolean): Dataset[T] def filter(conditionExpr: String): Dataset[T] def

Re: Questions about DataFrame's filter()

2016-09-29 Thread Michael Armbrust
-dev +user It surprises me as `filter()` takes a Column, not a `Row => Boolean`. There are several overloaded versions of Dataset.filter(...) def filter(func: FilterFunction[T]): Dataset[T] def filter(func: (T) ⇒ Boolean): Dataset[T] def filter(conditionExpr: String): Dataset[T] def

Re: Spark SQL JSON Column Support

2016-09-29 Thread Michael Armbrust
> > Will this be able to handle projection pushdown if a given job doesn't > utilize all the columns in the schema? Or should people have a per-job schema? > As currently written, we will do a little bit of extra work to pull out fields that aren't needed. I think it would be pretty straight

Re: [VOTE] Release Apache Spark 2.0.1 (RC4)

2016-09-29 Thread Michael Armbrust
+1 On Thu, Sep 29, 2016 at 11:51 AM, Mridul Muralidharan wrote: > +1 > > Regards, > Mridul > > On Wed, Sep 28, 2016 at 7:14 PM, Reynold Xin wrote: > > Please vote on releasing the following candidate as Apache Spark version > > 2.0.1. The vote is open

Re: Spark SQL JSON Column Support

2016-09-28 Thread Michael Armbrust
of life improvement for us. >> >> On Wed, Sep 28, 2016 at 10:52 AM, Michael Armbrust < >> mich...@databricks.com> wrote: >> >>> Spark SQL has great support for reading text files that contain JSON >>> data. However, in many cases the JSON data is

Re: Dataset doesn't have partitioner after a repartition on one of the columns

2016-09-28 Thread Michael Armbrust
Hi Darin, In SQL we have finer grained information about partitioning, so we don't use the RDD Partitioner. Here's a notebook that walks

Spark SQL JSON Column Support

2016-09-28 Thread Michael Armbrust
Spark SQL has great support for reading text files that contain JSON data. However, in many cases the JSON data is just one column amongst others. This is particularly true when reading from sources such as Kafka. This PR adds a new functions from_json

[jira] [Created] (SPARK-17699) from_json function for parsing json Strings into Structs

2016-09-27 Thread Michael Armbrust (JIRA)
Michael Armbrust created SPARK-17699: Summary: from_json function for parsing json Strings into Structs Key: SPARK-17699 URL: https://issues.apache.org/jira/browse/SPARK-17699 Project: Spark

[jira] [Resolved] (SPARK-17153) [Structured streams] readStream ignores partition columns

2016-09-26 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17153?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust resolved SPARK-17153. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request 14803

Re: Spark 2.0 Structured Streaming: sc.parallelize in foreach sink cause Task not serializable error

2016-09-26 Thread Michael Armbrust
The code in ForeachWriter runs on the executors, which means that you are not allowed to use the SparkContext. This is probably why you are seeing that exception. On Sun, Sep 25, 2016 at 3:20 PM, Jianshi wrote: > Dear all: > > I am trying out the new released feature of

Re: udf forces usage of Row for complex types?

2016-09-26 Thread Michael Armbrust
I agree this should work. We just haven't finished killing the old reflection based conversion logic now that we have more powerful/efficient encoders. Please open a JIRA. On Sun, Sep 25, 2016 at 2:41 PM, Koert Kuipers wrote: > after having gotten used to have case classes

[jira] [Created] (SPARK-17643) Remove comparable requirement from Offset

2016-09-22 Thread Michael Armbrust (JIRA)
Michael Armbrust created SPARK-17643: Summary: Remove comparable requirement from Offset Key: SPARK-17643 URL: https://issues.apache.org/jira/browse/SPARK-17643 Project: Spark Issue Type

[jira] [Assigned] (SPARK-17643) Remove comparable requirement from Offset

2016-09-22 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17643?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust reassigned SPARK-17643: Assignee: Michael Armbrust > Remove comparable requirement from Off

[jira] [Created] (SPARK-17627) Streaming Providers should be labeled Experimental

2016-09-21 Thread Michael Armbrust (JIRA)
Michael Armbrust created SPARK-17627: Summary: Streaming Providers should be labeled Experimental Key: SPARK-17627 URL: https://issues.apache.org/jira/browse/SPARK-17627 Project: Spark

[jira] [Updated] (SPARK-17627) Streaming Providers should be labeled Experimental

2016-09-21 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17627?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust updated SPARK-17627: - Component/s: SQL > Streaming Providers should be labeled Experimen

[jira] [Commented] (SPARK-16407) Allow users to supply custom StreamSinkProviders

2016-09-21 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-16407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15510787#comment-15510787 ] Michael Armbrust commented on SPARK-16407: -- I'm still a little unclear on the use cases we

[jira] [Commented] (SPARK-16407) Allow users to supply custom StreamSinkProviders

2016-09-19 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-16407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15504301#comment-15504301 ] Michael Armbrust commented on SPARK-16407: -- You are taking an experimental interface, which

[jira] [Commented] (SPARK-16407) Allow users to supply custom StreamSinkProviders

2016-09-19 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-16407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15504303#comment-15504303 ] Michael Armbrust commented on SPARK-16407: -- You are taking an experimental interface, which

Re: Spark SQL - Applying transformation on a struct inside an array

2016-09-15 Thread Michael Armbrust
t; I'm currently trying to create a generic transformation mecanism on a >> Dataframe to modify an arbitrary column regardless of the underlying the >> schema. >> >> It's "relatively" straightforward for complex types like >> struct<struct<…>> to ap

[jira] [Commented] (SPARK-16407) Allow users to supply custom StreamSinkProviders

2016-09-15 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-16407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15494475#comment-15494475 ] Michael Armbrust commented on SPARK-16407: -- Sure, but the bar for compatibility is different

[jira] [Comment Edited] (SPARK-16407) Allow users to supply custom StreamSinkProviders

2016-09-15 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-16407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15494475#comment-15494475 ] Michael Armbrust edited comment on SPARK-16407 at 9/15/16 8:42 PM

[jira] [Commented] (SPARK-16407) Allow users to supply custom StreamSinkProviders

2016-09-14 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-16407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15491819#comment-15491819 ] Michael Armbrust commented on SPARK-16407: -- I think it is likely that we will want to make

[jira] [Updated] (SPARK-17445) Reference an ASF page as the main place to find third-party packages

2016-09-13 Thread Michael Armbrust (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-17445?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Armbrust updated SPARK-17445: - Target Version/s: 2.0.1, 2.1.0 > Reference an ASF page as the main place to find th

<    2   3   4   5   6   7   8   9   10   11   >