Re: uncontinuous offset in kafka will cause the spark streaming failure

2018-01-23 Thread Justin Miller
We appear to be kindred spirits, I’ve recently run into the same issue. Are you 
running compacted topics? I’ve run into this issue on non-compacted topics as 
well, it happens rarely but is still a pain. You might check out this patch and 
related spark streaming Kafka ticket:

https://github.com/apache/spark/compare/master...koeninger:SPARK-17147 

https://issues.apache.org/jira/browse/SPARK-17147 


I’ll be testing out the patch on somewhat large scale stream processor tomorrow.

CCing: Cody Koeninger

Best,
Justin

> On Jan 23, 2018, at 10:48 PM, namesuperwood  wrote:
> 
> Hi all
> 
>   kafka version :  kafka_2.11-0.11.0.2
>spark version :  2.0.1 
> 
>   A topic-partition "adn-tracking,15"  in kafka  who's   earliest offset 
> is  1255644602 and  latest offset is 1271253441.  
>   While starting a spark streaming to process the data from the topic ,  
> we got a exception with "Got wrong record   even after seeking to offset 
> 1266921577”.  [   (earliest offset) 1255644602 < 1266921577   < 
> 1271253441 ( latest offset ) ]
>   Finally, I found the following source code in class 
> CachedKafkaCounsumer from spark-streaming. This is obviously due to the fact 
> that the offset from consumer poll and the offset which the comsuner seek is 
> not equal.
>   
>   Here is the “ CachedKafkaCounsumer.scala” code:
> 
>   def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = { 
> 
>   logDebug(s"Get $groupId $topic $partition nextOffset $nextOffset 
> requested $offset") if (offset !=  nextOffset) { 
> 
>   logInfo(s"Initial fetch for $groupId $topic $partition $offset") 
> seek(offset) poll(timeout) }
> 
> if (!buffer.hasNext()) { poll(timeout) }
> assert(buffer.hasNext(),
>   s"Failed to get records for $groupId $topic $partition $offset after 
> polling for $timeout")
> var record = buffer.next()
> 
> if (record.offset != offset) {
>   logInfo(s"Buffer miss for $groupId $topic $partition $offset")
>   seek(offset)
>   poll(timeout)
>   assert(buffer.hasNext(),
> s"Failed to get records for $groupId $topic $partition $offset after 
> polling for $timeout")
>   record = buffer.next()
>   assert(record.offset == offset,
> s"Got wrong record for $groupId $topic $partition even after seeking to 
> offset $offset")
> }
> 
> nextOffset = offset + 1
> record 
> }
>   I reproduce this problem, and found out that offset from one 
> topicAndPartition is uncontinuous in Kafka。I think this is a bug that needs 
> to be repaired.
>   I  implemented a simple project to use consumer to  seek offset 
> 1266921577. But it return the offset 1266921578. Then while  seek to 
> 1266921576, it return the 1266921576 exactly。 
>   
>   
> 
> 
> There is the code:
> public class consumerDemo {
> public static void main(String[] argv){   
>   Properties props = new Properties(); 
>   props.put("bootstrap.servers", "172.31.29.31:9091"); 
>   props.put("group.id", "consumer-tutorial-demo"); 
>   props.put("key.deserializer", StringDeserializer.class.getName()); 
>   props.put("value.deserializer", StringDeserializer.class.getName()); 
>   KafkaConsumer consumer = new KafkaConsumer String>(props); 
>   TopicPartition tp = new TopicPartition("adn-tracking-click", 15); 
>   Collection collection = new 
> ArrayList(); 
>   collection.add(tp); consumer.assign(collection); 
>   consumer.seek(tp, 1266921576); ConsumerRecords 
> consumerRecords = consumer.poll(1); 
>   List> listR = 
> consumerRecords.records(tp);
>   Iterator > iter = listR.iterator(); 
>   ConsumerRecord record = iter.next(); 
>   System.out.println(" the next record " + record.offset() + " recode 
> topic " + record.topic());
>}
> }
> 
> 
> 
> 
> 
> 
> 
> 
> 
> wood.super



Re: "Got wrong record after seeking to offset" issue

2018-01-18 Thread Justin Miller
Yeah I saw that after I sent that e-mail out. Iactually remembered another 
ticket that I had commented on that turned out to be unrelated to the issue I 
was seeing at the time. It may be related to the current issue:

https://issues.apache.org/jira/browse/SPARK-17147 
<https://issues.apache.org/jira/browse/SPARK-17147>

We are compacting topics, but only offset topics. We just updated our message 
version to 0.10 today as our last non-Spark project was brought up to 0.11 
(Storm based).

Justin

> On Jan 18, 2018, at 1:39 PM, Cody Koeninger <c...@koeninger.org> wrote:
> 
> https://kafka.apache.org/documentation/#compaction
> 
> On Thu, Jan 18, 2018 at 1:17 AM, Justin Miller
> <justin.mil...@protectwise.com> wrote:
>> By compacted do you mean compression? If so then we did recently turn on lz4
>> compression. If there’s another meaning if there’s a command I can run to
>> check compaction I’m happy to give that a shot too.
>> 
>> I’ll try consuming from the failed offset if/when the problem manifests
>> itself again.
>> 
>> Thanks!
>> Justin
>> 
>> 
>> On Wednesday, January 17, 2018, Cody Koeninger <c...@koeninger.org> wrote:
>>> 
>>> That means the consumer on the executor tried to seek to the specified
>>> offset, but the message that was returned did not have a matching
>>> offset.  If the executor can't get the messages the driver told it to
>>> get, something's generally wrong.
>>> 
>>> What happens when you try to consume the particular failing offset
>>> from another  (e.g. commandline) consumer?
>>> 
>>> Is the topic in question compacted?
>>> 
>>> 
>>> 
>>> On Tue, Jan 16, 2018 at 11:10 PM, Justin Miller
>>> <justin.mil...@protectwise.com> wrote:
>>>> Greetings all,
>>>> 
>>>> I’ve recently started hitting on the following error in Spark Streaming
>>>> in Kafka. Adjusting maxRetries and spark.streaming.kafka.consumer.poll.ms
>>>> even to five minutes doesn’t seem to be helping. The problem only 
>>>> manifested
>>>> in the last few days, restarting with a new consumer group seems to remedy
>>>> the issue for a few hours (< retention, which is 12 hours).
>>>> 
>>>> Error:
>>>> Caused by: java.lang.AssertionError: assertion failed: Got wrong record
>>>> for spark-executor-  76 even after seeking to
>>>> offset 1759148155
>>>>at scala.Predef$.assert(Predef.scala:170)
>>>>at
>>>> org.apache.spark.streaming.kafka010.CachedKafkaConsumer.get(CachedKafkaConsumer.scala:85)
>>>>at
>>>> org.apache.spark.streaming.kafka010.KafkaRDD$KafkaRDDIterator.next(KafkaRDD.scala:223)
>>>>at
>>>> org.apache.spark.streaming.kafka010.KafkaRDD$KafkaRDDIterator.next(KafkaRDD.scala:189)
>>>>at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
>>>> 
>>>> I guess my questions are, why is that assertion a job killer vs a
>>>> warning and is there anything I can tweak settings wise that may keep it at
>>>> bay.
>>>> 
>>>> I wouldn’t be surprised if this issue were exacerbated by the volume we
>>>> do on Kafka topics (~150k/sec on the persister that’s crashing).
>>>> 
>>>> Thank you!
>>>> Justin
>>>> 
>>>> 
>>>> -
>>>> To unsubscribe e-mail: user-unsubscr...@spark.apache.org
>>>> 



Re: "Got wrong record after seeking to offset" issue

2018-01-17 Thread Justin Miller
By compacted do you mean compression? If so then we did recently turn on
lz4 compression. If there’s another meaning if there’s a command I can run
to check compaction I’m happy to give that a shot too.

I’ll try consuming from the failed offset if/when the problem manifests
itself again.

Thanks!
Justin

On Wednesday, January 17, 2018, Cody Koeninger <c...@koeninger.org> wrote:

> That means the consumer on the executor tried to seek to the specified
> offset, but the message that was returned did not have a matching
> offset.  If the executor can't get the messages the driver told it to
> get, something's generally wrong.
>
> What happens when you try to consume the particular failing offset
> from another  (e.g. commandline) consumer?
>
> Is the topic in question compacted?
>
>
>
> On Tue, Jan 16, 2018 at 11:10 PM, Justin Miller
> <justin.mil...@protectwise.com> wrote:
> > Greetings all,
> >
> > I’ve recently started hitting on the following error in Spark Streaming
> in Kafka. Adjusting maxRetries and spark.streaming.kafka.consumer.poll.ms
> even to five minutes doesn’t seem to be helping. The problem only
> manifested in the last few days, restarting with a new consumer group seems
> to remedy the issue for a few hours (< retention, which is 12 hours).
> >
> > Error:
> > Caused by: java.lang.AssertionError: assertion failed: Got wrong record
> for spark-executor-  76 even after seeking
> to offset 1759148155
> > at scala.Predef$.assert(Predef.scala:170)
> > at org.apache.spark.streaming.kafka010.CachedKafkaConsumer.
> get(CachedKafkaConsumer.scala:85)
> > at org.apache.spark.streaming.kafka010.KafkaRDD$
> KafkaRDDIterator.next(KafkaRDD.scala:223)
> > at org.apache.spark.streaming.kafka010.KafkaRDD$
> KafkaRDDIterator.next(KafkaRDD.scala:189)
> > at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
> >
> > I guess my questions are, why is that assertion a job killer vs a
> warning and is there anything I can tweak settings wise that may keep it at
> bay.
> >
> > I wouldn’t be surprised if this issue were exacerbated by the volume we
> do on Kafka topics (~150k/sec on the persister that’s crashing).
> >
> > Thank you!
> > Justin
> >
> >
> > -
> > To unsubscribe e-mail: user-unsubscr...@spark.apache.org
> >
>


"Got wrong record after seeking to offset" issue

2018-01-16 Thread Justin Miller
Greetings all,

I’ve recently started hitting on the following error in Spark Streaming in 
Kafka. Adjusting maxRetries and spark.streaming.kafka.consumer.poll.ms even to 
five minutes doesn’t seem to be helping. The problem only manifested in the 
last few days, restarting with a new consumer group seems to remedy the issue 
for a few hours (< retention, which is 12 hours).

Error:
Caused by: java.lang.AssertionError: assertion failed: Got wrong record for 
spark-executor-  76 even after seeking to offset 
1759148155
at scala.Predef$.assert(Predef.scala:170)
at 
org.apache.spark.streaming.kafka010.CachedKafkaConsumer.get(CachedKafkaConsumer.scala:85)
at 
org.apache.spark.streaming.kafka010.KafkaRDD$KafkaRDDIterator.next(KafkaRDD.scala:223)
at 
org.apache.spark.streaming.kafka010.KafkaRDD$KafkaRDDIterator.next(KafkaRDD.scala:189)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)

I guess my questions are, why is that assertion a job killer vs a warning and 
is there anything I can tweak settings wise that may keep it at bay.

I wouldn’t be surprised if this issue were exacerbated by the volume we do on 
Kafka topics (~150k/sec on the persister that’s crashing).

Thank you!
Justin


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



Forcing either Hive or Spark SQL representation for metastore

2017-05-18 Thread Justin Miller
Hello,

I was wondering if there were a way to force one representation or another for 
the Hive metastore. Some of our data can’t be parsed with the Hive method so it 
switches over to the Spark SQL method, leaving some of our data stored in Spark 
SQL format and some in Hive format. It’d be nice if we could force it to use 
the Spark SQL format, as changing the underlying data would be difficult.

Some have (Spark SQL): 

17/05/18 21:50:29 WARN HiveExternalCatalog: Could not persist 
`default`.`tablenamehere` in a Hive compatible way. Persisting it into Hive 
metastore in Spark SQL specific format.

Some have (Hive):

17/05/18 21:52:27 INFO CatalystSqlParser: Parsing command: int
17/05/18 21:52:27 INFO CatalystSqlParser: Parsing command: int
17/05/18 21:52:27 INFO CatalystSqlParser: Parsing command: int
17/05/18 21:52:27 INFO CatalystSqlParser: Parsing command: int
17/05/18 21:52:27 INFO CatalystSqlParser: Parsing command: bigint
17/05/18 21:52:27 INFO CatalystSqlParser: Parsing command: 
struct

Another question about persisting to S3, I’m getting the following for all: 

Caused by: MetaException(message:java.io.IOException: Got exception: 
java.io.IOException 
/username/sys_encrypted/staging/raw/updatedTimeYear=2017/updatedTimeMonth=5/updatedTimeDay=16/updatedTimeHour=23
 doesn't exist)

Thanks!
Justin

Spark Streaming Kafka Job has strange behavior for certain tasks

2017-04-05 Thread Justin Miller
Greetings!

I've been running various spark streaming jobs to persist data from kafka 
topics and one persister in particular seems to have issues. I've verified that 
the number of messages is the same per partition (roughly of course) and the 
volume of data is a fraction of the volume of other persisters that appear to 
be working fine. 

The tasks appear to go fine until approximately 74-80 of the tasks (of 96) in, 
and then the remaining tasks take a while. I'm using EMR/Spark 2.1.0/Kafka 
0.10.0.1/EMRFS (EMR's S3 solution). Any help would be greatly appreciated!

Here's the code I'm using to do the transformation:

val transformedData = transformer(sqlContext.createDataFrame(values, 
converter.schema))

transformedData
  .write
  .mode(Append)
  .partitionBy(persisterConfig.partitioning: _*)
  .format("parquet")
  .save(parquetPath)

Here's the output of the job as it's running (thrift -> parquet/snappy -> s3 is 
the flow), the files are roughly the same size (96 files per 10 minute window):

17/04/05 16:43:43 INFO TaskSetManager: Finished task 72.0 in stage 7.0 (TID 
722) in 10089 ms on ip-172-20-213-64.us-west-2.compute.internal (executor 57) 
(1/96)
17/04/05 16:43:43 INFO TaskSetManager: Finished task 58.0 in stage 7.0 (TID 
680) in 10099 ms on ip-172-20-218-229.us-west-2.compute.internal (executor 90) 
(2/96)
17/04/05 16:43:43 INFO TaskSetManager: Finished task 81.0 in stage 7.0 (TID 
687) in 10244 ms on ip-172-20-218-144.us-west-2.compute.internal (executor 8) 
(3/96)
17/04/05 16:43:43 INFO TaskSetManager: Finished task 23.0 in stage 7.0 (TID 
736) in 10236 ms on ip-172-20-209-248.us-west-2.compute.internal (executor 82) 
(4/96)
17/04/05 16:43:43 INFO TaskSetManager: Finished task 52.0 in stage 7.0 (TID 
730) in 10275 ms on ip-172-20-218-144.us-west-2.compute.internal (executor 78) 
(5/96)
17/04/05 16:43:43 INFO TaskSetManager: Finished task 45.0 in stage 7.0 (TID 
691) in 10289 ms on ip-172-20-215-172.us-west-2.compute.internal (executor 41) 
(6/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 13.0 in stage 7.0 (TID 
712) in 10532 ms on ip-172-20-223-100.us-west-2.compute.internal (executor 65) 
(7/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 42.0 in stage 7.0 (TID 
694) in 10595 ms on ip-172-20-208-230.us-west-2.compute.internal (executor 18) 
(8/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 2.0 in stage 7.0 (TID 763) 
in 10623 ms on ip-172-20-208-230.us-west-2.compute.internal (executor 74) (9/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 82.0 in stage 7.0 (TID 
727) in 10631 ms on ip-172-20-212-76.us-west-2.compute.internal (executor 72) 
(10/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 69.0 in stage 7.0 (TID 
729) in 10716 ms on ip-172-20-215-172.us-west-2.compute.internal (executor 55) 
(11/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 65.0 in stage 7.0 (TID 
673) in 10733 ms on ip-172-20-217-201.us-west-2.compute.internal (executor 67) 
(12/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 15.0 in stage 7.0 (TID 
684) in 10737 ms on ip-172-20-213-64.us-west-2.compute.internal (executor 85) 
(13/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 27.0 in stage 7.0 (TID 
748) in 10747 ms on ip-172-20-217-201.us-west-2.compute.internal (executor 10) 
(14/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 46.0 in stage 7.0 (TID 
699) in 10834 ms on ip-172-20-218-229.us-west-2.compute.internal (executor 48) 
(15/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 6.0 in stage 7.0 (TID 719) 
in 10838 ms on ip-172-20-211-125.us-west-2.compute.internal (executor 52) 
(16/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 11.0 in stage 7.0 (TID 
739) in 10892 ms on ip-172-20-215-172.us-west-2.compute.internal (executor 83) 
(17/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 88.0 in stage 7.0 (TID 
697) in 10900 ms on ip-172-20-212-43.us-west-2.compute.internal (executor 70) 
(18/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 35.0 in stage 7.0 (TID 
678) in 10909 ms on ip-172-20-212-63.us-west-2.compute.internal (executor 77) 
(19/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 700) 
in 10906 ms on ip-172-20-208-230.us-west-2.compute.internal (executor 46) 
(20/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 36.0 in stage 7.0 (TID 
732) in 10935 ms on ip-172-20-215-172.us-west-2.compute.internal (executor 69) 
(21/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 19.0 in stage 7.0 (TID 
759) in 10948 ms on ip-172-20-223-100.us-west-2.compute.internal (executor 37) 
(22/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 41.0 in stage 7.0 (TID 
703) in 11013 ms on ip-172-20-217-201.us-west-2.compute.internal (executor 81) 
(23/96)
17/04/05 16:43:44 INFO TaskSetManager: Finished task 8.0 in stage 7.0 (TID 745) 
in 11007 ms on ip-172-20-215-172.us-west-2.compute.internal (executor 13) 
(24/96)
17/04/05 16:43:44 INFO TaskSetManager: 

Re: How to gracefully handle Kafka OffsetOutOfRangeException

2017-03-10 Thread Justin Miller
I've created a ticket here: https://issues.apache.org/jira/browse/SPARK-19888 
<https://issues.apache.org/jira/browse/SPARK-19888>

Thanks,
Justin

> On Mar 10, 2017, at 1:14 PM, Michael Armbrust <mich...@databricks.com> wrote:
> 
> If you have a reproduction you should open a JIRA.  It would be great if 
> there is a fix.  I'm just saying I know a similar issue does not exist in 
> structured streaming.
> 
> On Fri, Mar 10, 2017 at 7:46 AM, Justin Miller <justin.mil...@protectwise.com 
> <mailto:justin.mil...@protectwise.com>> wrote:
> Hi Michael,
> 
> I'm experiencing a similar issue. Will this not be fixed in Spark Streaming?
> 
> Best,
> Justin
> 
>> On Mar 10, 2017, at 8:34 AM, Michael Armbrust <mich...@databricks.com 
>> <mailto:mich...@databricks.com>> wrote:
>> 
>> One option here would be to try Structured Streaming.  We've added an option 
>> "failOnDataLoss" that will cause Spark to just skip a head when this 
>> exception is encountered (its off by default though so you don't silently 
>> miss data).
>> 
>> On Fri, Mar 18, 2016 at 4:16 AM, Ramkumar Venkataraman 
>> <ram.the.m...@gmail.com <mailto:ram.the.m...@gmail.com>> wrote:
>> I am using Spark streaming and reading data from Kafka using
>> KafkaUtils.createDirectStream. I have the "auto.offset.reset" set to
>> smallest.
>> 
>> But in some Kafka partitions, I get kafka.common.OffsetOutOfRangeException
>> and my spark job crashes.
>> 
>> I want to understand if there is a graceful way to handle this failure and
>> not kill the job. I want to keep ignoring these exceptions, as some other
>> partitions are fine and I am okay with data loss.
>> 
>> Is there any way to handle this and not have my spark job crash? I have no
>> option of increasing the kafka retention period.
>> 
>> I tried to have the DStream returned by createDirectStream() wrapped in a
>> Try construct, but since the exception happens in the executor, the Try
>> construct didn't take effect. Do you have any ideas of how to handle this?
>> 
>> 
>> 
>> --
>> View this message in context: 
>> http://apache-spark-user-list.1001560.n3.nabble.com/How-to-gracefully-handle-Kafka-OffsetOutOfRangeException-tp26534.html
>>  
>> <http://apache-spark-user-list.1001560.n3.nabble.com/How-to-gracefully-handle-Kafka-OffsetOutOfRangeException-tp26534.html>
>> Sent from the Apache Spark User List mailing list archive at Nabble.com 
>> <http://nabble.com/>.
>> 
>> -
>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org 
>> <mailto:user-unsubscr...@spark.apache.org>
>> For additional commands, e-mail: user-h...@spark.apache.org 
>> <mailto:user-h...@spark.apache.org>
>> 
>> 
> 
> 



Re: How to gracefully handle Kafka OffsetOutOfRangeException

2017-03-10 Thread Justin Miller
Hi Michael,

I'm experiencing a similar issue. Will this not be fixed in Spark Streaming?

Best,
Justin

> On Mar 10, 2017, at 8:34 AM, Michael Armbrust  wrote:
> 
> One option here would be to try Structured Streaming.  We've added an option 
> "failOnDataLoss" that will cause Spark to just skip a head when this 
> exception is encountered (its off by default though so you don't silently 
> miss data).
> 
> On Fri, Mar 18, 2016 at 4:16 AM, Ramkumar Venkataraman 
> > wrote:
> I am using Spark streaming and reading data from Kafka using
> KafkaUtils.createDirectStream. I have the "auto.offset.reset" set to
> smallest.
> 
> But in some Kafka partitions, I get kafka.common.OffsetOutOfRangeException
> and my spark job crashes.
> 
> I want to understand if there is a graceful way to handle this failure and
> not kill the job. I want to keep ignoring these exceptions, as some other
> partitions are fine and I am okay with data loss.
> 
> Is there any way to handle this and not have my spark job crash? I have no
> option of increasing the kafka retention period.
> 
> I tried to have the DStream returned by createDirectStream() wrapped in a
> Try construct, but since the exception happens in the executor, the Try
> construct didn't take effect. Do you have any ideas of how to handle this?
> 
> 
> 
> --
> View this message in context: 
> http://apache-spark-user-list.1001560.n3.nabble.com/How-to-gracefully-handle-Kafka-OffsetOutOfRangeException-tp26534.html
>  
> 
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
> 
> -
> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org 
> 
> For additional commands, e-mail: user-h...@spark.apache.org 
> 
> 
> 



Re: Is there any scheduled release date for Spark 2.1.0?

2016-12-28 Thread Justin Miller
Interesting, because a bug that seemed to be fixed in 2.1.0-SNAPSHOT doesn't 
appear to be fixed in 2.1.0 stable (it centered around a null-pointer exception 
during code gen). It seems to be fixed in 2.1.1-SNAPSHOT, but I can try stable 
again.

> On Dec 28, 2016, at 1:38 PM, Mark Hamstra <m...@clearstorydata.com> wrote:
> 
> A SNAPSHOT build is not a stable artifact, but rather floats to the top of 
> commits that are intended for the next release.  So, 2.1.1-SNAPSHOT comes 
> after the 2.1.0 release and contains any code at the time that the artifact 
> was built that was committed to the branch-2.1 maintenance branch and is, 
> therefore, intended for the eventual 2.1.1 maintenance release.  Once a 
> release is tagged and stable artifacts for it can be built, there is no 
> purpose for s SNAPSHOT of that release -- e.g. there is no longer any purpose 
> for a 2.1.0-SNAPSHOT release; if you want 2.1.0, then you should be using 
> stable artifacts now, not SNAPSHOTs.
> 
> The existence of a SNAPSHOT doesn't imply anything about the release date of 
> the associated finished version.  Rather, it only indicates a name that is 
> attached to all of the code that is currently intended for the associated 
> release number. 
> 
> On Wed, Dec 28, 2016 at 3:09 PM, Justin Miller <justin.mil...@protectwise.com 
> <mailto:justin.mil...@protectwise.com>> wrote:
> It looks like the jars for 2.1.0-SNAPSHOT are gone?
> 
> https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.1.0-SNAPSHOT/
>  
> <https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.1.0-SNAPSHOT/>
> 
> Also:
> 
> 2.1.0-SNAPSHOT/ 
> <https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.1.0-SNAPSHOT/>
>   Fri Dec 23 16:31:42 UTC 2016
> 2.1.1-SNAPSHOT/ 
> <https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.1.1-SNAPSHOT/>
>   Wed Dec 28 20:01:10 UTC 2016
> 2.2.0-SNAPSHOT/ 
> <https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.2.0-SNAPSHOT/>
>   Wed Dec 28 19:12:38 UTC 2016 
> 
> What's with 2.1.1-SNAPSHOT? Is that version about to be released as well?
> 
> Thanks!
> Justin
> 
>> On Dec 28, 2016, at 12:53 PM, Mark Hamstra <m...@clearstorydata.com 
>> <mailto:m...@clearstorydata.com>> wrote:
>> 
>> The v2.1.0 tag is there: https://github.com/apache/spark/tree/v2.1.0 
>> <https://github.com/apache/spark/tree/v2.1.0>
>> 
>> On Wed, Dec 28, 2016 at 2:04 PM, Koert Kuipers <ko...@tresata.com 
>> <mailto:ko...@tresata.com>> wrote:
>> seems like the artifacts are on maven central but the website is not yet 
>> updated.
>> 
>> strangely the tag v2.1.0 is not yet available on github. i assume its equal 
>> to v2.1.0-rc5
>> 
>> On Fri, Dec 23, 2016 at 10:52 AM, Justin Miller 
>> <justin.mil...@protectwise.com <mailto:justin.mil...@protectwise.com>> wrote:
>> I'm curious about this as well. Seems like the vote passed.
>> 
>> > On Dec 23, 2016, at 2:00 AM, Aseem Bansal <asmbans...@gmail.com 
>> > <mailto:asmbans...@gmail.com>> wrote:
>> >
>> >
>> 
>> 
>> -
>> To unsubscribe e-mail: user-unsubscr...@spark.apache.org 
>> <mailto:user-unsubscr...@spark.apache.org>
>> 
>> 
>> 
> 
> 



Re: Is there any scheduled release date for Spark 2.1.0?

2016-12-28 Thread Justin Miller
It looks like the jars for 2.1.0-SNAPSHOT are gone?

https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.1.0-SNAPSHOT/
 
<https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.1.0-SNAPSHOT/>

Also:

2.1.0-SNAPSHOT/ 
<https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.1.0-SNAPSHOT/>
Fri Dec 23 16:31:42 UTC 2016
2.1.1-SNAPSHOT/ 
<https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.1.1-SNAPSHOT/>
Wed Dec 28 20:01:10 UTC 2016
2.2.0-SNAPSHOT/ 
<https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.2.0-SNAPSHOT/>
Wed Dec 28 19:12:38 UTC 2016 

What's with 2.1.1-SNAPSHOT? Is that version about to be released as well?

Thanks!
Justin

> On Dec 28, 2016, at 12:53 PM, Mark Hamstra <m...@clearstorydata.com> wrote:
> 
> The v2.1.0 tag is there: https://github.com/apache/spark/tree/v2.1.0 
> <https://github.com/apache/spark/tree/v2.1.0>
> 
> On Wed, Dec 28, 2016 at 2:04 PM, Koert Kuipers <ko...@tresata.com 
> <mailto:ko...@tresata.com>> wrote:
> seems like the artifacts are on maven central but the website is not yet 
> updated.
> 
> strangely the tag v2.1.0 is not yet available on github. i assume its equal 
> to v2.1.0-rc5
> 
> On Fri, Dec 23, 2016 at 10:52 AM, Justin Miller 
> <justin.mil...@protectwise.com <mailto:justin.mil...@protectwise.com>> wrote:
> I'm curious about this as well. Seems like the vote passed.
> 
> > On Dec 23, 2016, at 2:00 AM, Aseem Bansal <asmbans...@gmail.com 
> > <mailto:asmbans...@gmail.com>> wrote:
> >
> >
> 
> 
> -
> To unsubscribe e-mail: user-unsubscr...@spark.apache.org 
> <mailto:user-unsubscr...@spark.apache.org>
> 
> 
> 



Re: Is there any scheduled release date for Spark 2.1.0?

2016-12-23 Thread Justin Miller
I'm curious about this as well. Seems like the vote passed. 

> On Dec 23, 2016, at 2:00 AM, Aseem Bansal  wrote:
> 
> 


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