SQL TIMESTAMP semantics vs. SPARK-18350

2017-05-24 Thread Zoltan Ivanfi
Hi, Sorry if you receive this mail twice, it seems that my first attempt did not make it to the list for some reason. I would like to start a discussion about SPARK-18350 before it gets released because it seems to be going in a different direct

Re: Mesos checkpointing

2017-05-24 Thread Michael Gummelt
> We had investigated internally recently why restarting the mesos agents failed the spark jobs (no real reason they should, right?) and came across the data. Restarting the agent without checkpointing enabled will kill the executor, but that still shouldn't cause the Spark job to fail, since Spar

Re: Mesos checkpointing

2017-05-24 Thread Charles Allen
The issue on our side is we tend to roll out a bunch of agent updates at about the same time. So rolling an agent, then waiting for spark jobs to recover, then rolling another agent is not at all practical. It is a huge benefit if we can just update the agents in bulk (or even sequentially, but onl

Re: Mesos checkpointing

2017-05-24 Thread Michael Gummelt
Ah, then yea, checkpointing should solve your problem. Let's do that. On Wed, May 24, 2017 at 11:19 AM, Charles Allen < charles.al...@metamarkets.com> wrote: > The issue on our side is we tend to roll out a bunch of agent updates at > about the same time. So rolling an agent, then waiting for sp

Running into the same problem as JIRA SPARK-19268

2017-05-24 Thread kant kodali
Hi All, I am using Spark 2.1.1 and running in a Standalone mode using HDFS and Kafka I am running into the same problem as https://issues.apache.org/jira/browse/SPARK-19268 with my app(not KafkaWordCount). Here is my sample code *Here is how I create ReadStream* sparkSession.readStream()

Re: Running into the same problem as JIRA SPARK-19268

2017-05-24 Thread kant kodali
Even if I do simple count aggregation like below I get the same error as https://issues.apache.org/jira/browse/SPARK-19268 Dataset df2 = df1.groupBy(functions.window(df1.col("Timestamp5"), "24 hours", "24 hours"), df1.col("AppName")).count(); On Wed, May 24, 2017 at 3:35 PM, kant kodali wrote:

Re: Running into the same problem as JIRA SPARK-19268

2017-05-24 Thread Michael Armbrust
-dev Have you tried clearing out the checkpoint directory? Can you also give the full stack trace? On Wed, May 24, 2017 at 3:45 PM, kant kodali wrote: > Even if I do simple count aggregation like below I get the same error as > https://issues.apache.org/jira/browse/SPARK-19268 > > Dataset df2

Crowdsourced triage Scapegoat compiler plugin warnings

2017-05-24 Thread Josh Rosen
I'm interested in using the Scapegoat Scala compiler plugin to find potential bugs and performance problems in Spark. Scapegoat has a useful built-in set of inspections and is pretty easy to extend with custom ones. For example, I added an inspection to spot

Re: SQL TIMESTAMP semantics vs. SPARK-18350

2017-05-24 Thread Ofir Manor
Hi Zoltan, thanks for bringing this up, this is really important to me! Personally, as a user developing app on top of Spark and other tools, the current timestamp semantics has been a source of some pain - needing to undo Spark's "auto-correcting" of timestamps . It would be really great if we cou