Re: Beam SparkRunner and Spark KryoSerializer problem

2018-10-04 Thread Juan Carlos Garcia
Hi Jean,

Thank you!

On Thu, Oct 4, 2018 at 7:54 AM Jean-Baptiste Onofré  wrote:

> Hi Juan
>
> I'm on it.
>
> Regards
> JB
> Le 4 oct. 2018, à 07:19, Juan Carlos Garcia  a écrit:
>>
>> Bump,
>>
>> can someone from the core-dev provide a feedback about:
>>https://issues.apache.org/jira/browse/BEAM-4597
>>
>> Thanks
>>
>> On Mon, Jul 30, 2018 at 3:15 PM Juan Carlos Garcia 
>> wrote:
>>
>>> Hi Jean,
>>>
>>> Thanks for taking a look.
>>>
>>>
>>> On Mon, Jul 30, 2018 at 2:49 PM, Jean-Baptiste Onofré 
>>> wrote:
>>>
 Hi Juan,

 it seems that has been introduce by the metrics layer in the core runner
 API.

 Let me check.

 Regards
 JB

 On 30/07/2018 14:47, Juan Carlos Garcia wrote:
 > Bump!
 >
 > Does any of the core-dev roam around here?
 >
 > Can someone provide a feedback about BEAM-4597
 > 
 >
 > Thanks and regards,
 >
 > On Thu, Jul 19, 2018 at 3:41 PM, Juan Carlos Garcia <
 jcgarc...@gmail.com
 > > wrote:
 >
 > Folks,
 >
 > Its someone using the SparkRunner out there with the Spark
 > KryoSerializer ?
 >
 > We are being force to use the not so efficient 'JavaSerializer'
 with
 > Spark because we face the following exception:
 >
 > 
 > Exception in thread "main" java.lang.RuntimeException:
 > org.apache.spark.SparkException: Job aborted due to stage failure:
 > Exception while getting task result:
 > com.esotericsoftware.kryo.KryoException: Unable to find class:
 >
  
 org.apache.beam.runners.core.metrics.MetricsContainerImpl$$Lambda$31/1875283985
 > Serialization trace:
 > factory (org.apache.beam.runners.core.metrics.MetricsMap)
 > counters
 (org.apache.beam.runners.core.metrics.MetricsContainerImpl)
 > metricsContainers
 > (org.apache.beam.runners.core.metrics.MetricsContainerStepMap)
 > metricsContainers
 > (org.apache.beam.runners.spark.io.SparkUnboundedSource$Metadata)
 > at
 >
  
 org.apache.beam.runners.spark.SparkPipelineResult.runtimeExceptionFrom(SparkPipelineResult.java:55)
 > at
 >
  
 org.apache.beam.runners.spark.SparkPipelineResult.beamExceptionFrom(SparkPipelineResult.java:72)
 > at
 >
  
 org.apache.beam.runners.spark.SparkPipelineResult.access$000(SparkPipelineResult.java:41)
 > at
 >
  
 org.apache.beam.runners.spark.SparkPipelineResult$StreamingMode.stop(SparkPipelineResult.java:163)
 > at
 >
  
 org.apache.beam.runners.spark.SparkPipelineResult.offerNewState(SparkPipelineResult.java:198)
 > at
 >
  
 org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:101)
 > at
 >
  
 org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:87)
 > at
 >
  
 org.apache.beam.examples.BugWithKryoOnSpark.main(BugWithKryoOnSpark.java:75)
 > 
 >
 > I created a jira ticket and attached a project example on it,
 > https://issues.apache.org/jira/browse/BEAM-4597
 > 
 >
 > Any feedback is appreciated.
 >
 > --
 >
 > JC
 >
 >
 >
 >
 > --
 >
 > JC
 >

 --
 Jean-Baptiste Onofré
 jbono...@apache.org
 http://blog.nanthrax.net
 Talend - http://www.talend.com

>>>
>>>
>>>
>>> --
>>>
>>> JC
>>>
>>>
>>
>> --
>>
>> JC
>>
>>

-- 

JC


Re: Beam SparkRunner and Spark KryoSerializer problem

2018-10-03 Thread Jean-Baptiste Onofré
Hi Juan

I'm on it.

Regards
JB

Le 4 oct. 2018 à 07:19, à 07:19, Juan Carlos Garcia  a 
écrit:
>Bump,
>
>can someone from the core-dev provide a feedback about:
>   https://issues.apache.org/jira/browse/BEAM-4597
>
>Thanks
>
>On Mon, Jul 30, 2018 at 3:15 PM Juan Carlos Garcia
>
>wrote:
>
>> Hi Jean,
>>
>> Thanks for taking a look.
>>
>>
>> On Mon, Jul 30, 2018 at 2:49 PM, Jean-Baptiste Onofré
>
>> wrote:
>>
>>> Hi Juan,
>>>
>>> it seems that has been introduce by the metrics layer in the core
>runner
>>> API.
>>>
>>> Let me check.
>>>
>>> Regards
>>> JB
>>>
>>> On 30/07/2018 14:47, Juan Carlos Garcia wrote:
>>> > Bump!
>>> >
>>> > Does any of the core-dev roam around here?
>>> >
>>> > Can someone provide a feedback about BEAM-4597
>>> > 
>>> >
>>> > Thanks and regards,
>>> >
>>> > On Thu, Jul 19, 2018 at 3:41 PM, Juan Carlos Garcia <
>>> jcgarc...@gmail.com
>>> > > wrote:
>>> >
>>> > Folks,
>>> >
>>> > Its someone using the SparkRunner out there with the Spark
>>> > KryoSerializer ?
>>> >
>>> > We are being force to use the not so efficient
>'JavaSerializer' with
>>> > Spark because we face the following exception:
>>> >
>>> > 
>>> > Exception in thread "main" java.lang.RuntimeException:
>>> > org.apache.spark.SparkException: Job aborted due to stage
>failure:
>>> > Exception while getting task result:
>>> > com.esotericsoftware.kryo.KryoException: Unable to find class:
>>> >
>>>
>org.apache.beam.runners.core.metrics.MetricsContainerImpl$$Lambda$31/1875283985
>>> > Serialization trace:
>>> > factory (org.apache.beam.runners.core.metrics.MetricsMap)
>>> > counters
>(org.apache.beam.runners.core.metrics.MetricsContainerImpl)
>>> > metricsContainers
>>> > (org.apache.beam.runners.core.metrics.MetricsContainerStepMap)
>>> > metricsContainers
>>> >   
>(org.apache.beam.runners.spark.io.SparkUnboundedSource$Metadata)
>>> > at
>>> >
>>>
>org.apache.beam.runners.spark.SparkPipelineResult.runtimeExceptionFrom(SparkPipelineResult.java:55)
>>> > at
>>> >
>>>
>org.apache.beam.runners.spark.SparkPipelineResult.beamExceptionFrom(SparkPipelineResult.java:72)
>>> > at
>>> >
>>>
>org.apache.beam.runners.spark.SparkPipelineResult.access$000(SparkPipelineResult.java:41)
>>> > at
>>> >
>>>
>org.apache.beam.runners.spark.SparkPipelineResult$StreamingMode.stop(SparkPipelineResult.java:163)
>>> > at
>>> >
>>>
>org.apache.beam.runners.spark.SparkPipelineResult.offerNewState(SparkPipelineResult.java:198)
>>> > at
>>> >
>>>
>org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:101)
>>> > at
>>> >
>>>
>org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:87)
>>> > at
>>> >
>>>
>org.apache.beam.examples.BugWithKryoOnSpark.main(BugWithKryoOnSpark.java:75)
>>> > 
>>> >
>>> > I created a jira ticket and attached a project example on it,
>>> > https://issues.apache.org/jira/browse/BEAM-4597
>>> > 
>>> >
>>> > Any feedback is appreciated.
>>> >
>>> > --
>>> >
>>> > JC
>>> >
>>> >
>>> >
>>> >
>>> > --
>>> >
>>> > JC
>>> >
>>>
>>> --
>>> Jean-Baptiste Onofré
>>> jbono...@apache.org
>>> http://blog.nanthrax.net
>>> Talend - http://www.talend.com
>>>
>>
>>
>>
>> --
>>
>> JC
>>
>>
>
>--
>
>JC


Re: Beam SparkRunner and Spark KryoSerializer problem

2018-10-03 Thread Juan Carlos Garcia
Bump,

can someone from the core-dev provide a feedback about:
   https://issues.apache.org/jira/browse/BEAM-4597

Thanks

On Mon, Jul 30, 2018 at 3:15 PM Juan Carlos Garcia 
wrote:

> Hi Jean,
>
> Thanks for taking a look.
>
>
> On Mon, Jul 30, 2018 at 2:49 PM, Jean-Baptiste Onofré 
> wrote:
>
>> Hi Juan,
>>
>> it seems that has been introduce by the metrics layer in the core runner
>> API.
>>
>> Let me check.
>>
>> Regards
>> JB
>>
>> On 30/07/2018 14:47, Juan Carlos Garcia wrote:
>> > Bump!
>> >
>> > Does any of the core-dev roam around here?
>> >
>> > Can someone provide a feedback about BEAM-4597
>> > 
>> >
>> > Thanks and regards,
>> >
>> > On Thu, Jul 19, 2018 at 3:41 PM, Juan Carlos Garcia <
>> jcgarc...@gmail.com
>> > > wrote:
>> >
>> > Folks,
>> >
>> > Its someone using the SparkRunner out there with the Spark
>> > KryoSerializer ?
>> >
>> > We are being force to use the not so efficient 'JavaSerializer' with
>> > Spark because we face the following exception:
>> >
>> > 
>> > Exception in thread "main" java.lang.RuntimeException:
>> > org.apache.spark.SparkException: Job aborted due to stage failure:
>> > Exception while getting task result:
>> > com.esotericsoftware.kryo.KryoException: Unable to find class:
>> >
>>  
>> org.apache.beam.runners.core.metrics.MetricsContainerImpl$$Lambda$31/1875283985
>> > Serialization trace:
>> > factory (org.apache.beam.runners.core.metrics.MetricsMap)
>> > counters (org.apache.beam.runners.core.metrics.MetricsContainerImpl)
>> > metricsContainers
>> > (org.apache.beam.runners.core.metrics.MetricsContainerStepMap)
>> > metricsContainers
>> > (org.apache.beam.runners.spark.io.SparkUnboundedSource$Metadata)
>> > at
>> >
>>  
>> org.apache.beam.runners.spark.SparkPipelineResult.runtimeExceptionFrom(SparkPipelineResult.java:55)
>> > at
>> >
>>  
>> org.apache.beam.runners.spark.SparkPipelineResult.beamExceptionFrom(SparkPipelineResult.java:72)
>> > at
>> >
>>  
>> org.apache.beam.runners.spark.SparkPipelineResult.access$000(SparkPipelineResult.java:41)
>> > at
>> >
>>  
>> org.apache.beam.runners.spark.SparkPipelineResult$StreamingMode.stop(SparkPipelineResult.java:163)
>> > at
>> >
>>  
>> org.apache.beam.runners.spark.SparkPipelineResult.offerNewState(SparkPipelineResult.java:198)
>> > at
>> >
>>  
>> org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:101)
>> > at
>> >
>>  
>> org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:87)
>> > at
>> >
>>  org.apache.beam.examples.BugWithKryoOnSpark.main(BugWithKryoOnSpark.java:75)
>> > 
>> >
>> > I created a jira ticket and attached a project example on it,
>> > https://issues.apache.org/jira/browse/BEAM-4597
>> > 
>> >
>> > Any feedback is appreciated.
>> >
>> > --
>> >
>> > JC
>> >
>> >
>> >
>> >
>> > --
>> >
>> > JC
>> >
>>
>> --
>> Jean-Baptiste Onofré
>> jbono...@apache.org
>> http://blog.nanthrax.net
>> Talend - http://www.talend.com
>>
>
>
>
> --
>
> JC
>
>

-- 

JC


Re: Beam SparkRunner and Spark KryoSerializer problem

2018-07-30 Thread Juan Carlos Garcia
Hi Jean,

Thanks for taking a look.


On Mon, Jul 30, 2018 at 2:49 PM, Jean-Baptiste Onofré 
wrote:

> Hi Juan,
>
> it seems that has been introduce by the metrics layer in the core runner
> API.
>
> Let me check.
>
> Regards
> JB
>
> On 30/07/2018 14:47, Juan Carlos Garcia wrote:
> > Bump!
> >
> > Does any of the core-dev roam around here?
> >
> > Can someone provide a feedback about BEAM-4597
> > 
> >
> > Thanks and regards,
> >
> > On Thu, Jul 19, 2018 at 3:41 PM, Juan Carlos Garcia  > > wrote:
> >
> > Folks,
> >
> > Its someone using the SparkRunner out there with the Spark
> > KryoSerializer ?
> >
> > We are being force to use the not so efficient 'JavaSerializer' with
> > Spark because we face the following exception:
> >
> > 
> > Exception in thread "main" java.lang.RuntimeException:
> > org.apache.spark.SparkException: Job aborted due to stage failure:
> > Exception while getting task result:
> > com.esotericsoftware.kryo.KryoException: Unable to find class:
> > org.apache.beam.runners.core.metrics.MetricsContainerImpl$$
> Lambda$31/1875283985
> > Serialization trace:
> > factory (org.apache.beam.runners.core.metrics.MetricsMap)
> > counters (org.apache.beam.runners.core.metrics.MetricsContainerImpl)
> > metricsContainers
> > (org.apache.beam.runners.core.metrics.MetricsContainerStepMap)
> > metricsContainers
> > (org.apache.beam.runners.spark.io.SparkUnboundedSource$Metadata)
> > at
> > org.apache.beam.runners.spark.SparkPipelineResult.
> runtimeExceptionFrom(SparkPipelineResult.java:55)
> > at
> > org.apache.beam.runners.spark.SparkPipelineResult.beamExceptionFrom(
> SparkPipelineResult.java:72)
> > at
> > org.apache.beam.runners.spark.SparkPipelineResult.access$
> 000(SparkPipelineResult.java:41)
> > at
> > org.apache.beam.runners.spark.SparkPipelineResult$
> StreamingMode.stop(SparkPipelineResult.java:163)
> > at
> > org.apache.beam.runners.spark.SparkPipelineResult.offerNewState(
> SparkPipelineResult.java:198)
> > at
> > org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(
> SparkPipelineResult.java:101)
> > at
> > org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(
> SparkPipelineResult.java:87)
> > at
> > org.apache.beam.examples.BugWithKryoOnSpark.main(
> BugWithKryoOnSpark.java:75)
> > 
> >
> > I created a jira ticket and attached a project example on it,
> > https://issues.apache.org/jira/browse/BEAM-4597
> > 
> >
> > Any feedback is appreciated.
> >
> > --
> >
> > JC
> >
> >
> >
> >
> > --
> >
> > JC
> >
>
> --
> Jean-Baptiste Onofré
> jbono...@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>



-- 

JC


Re: Beam SparkRunner and Spark KryoSerializer problem

2018-07-30 Thread Jean-Baptiste Onofré
Hi Juan,

it seems that has been introduce by the metrics layer in the core runner
API.

Let me check.

Regards
JB

On 30/07/2018 14:47, Juan Carlos Garcia wrote:
> Bump!
> 
> Does any of the core-dev roam around here?
> 
> Can someone provide a feedback about BEAM-4597
> 
> 
> Thanks and regards,
> 
> On Thu, Jul 19, 2018 at 3:41 PM, Juan Carlos Garcia  > wrote:
> 
> Folks,
> 
> Its someone using the SparkRunner out there with the Spark
> KryoSerializer ?
> 
> We are being force to use the not so efficient 'JavaSerializer' with
> Spark because we face the following exception:
> 
> 
> Exception in thread "main" java.lang.RuntimeException:
> org.apache.spark.SparkException: Job aborted due to stage failure:
> Exception while getting task result:
> com.esotericsoftware.kryo.KryoException: Unable to find class:
> 
> org.apache.beam.runners.core.metrics.MetricsContainerImpl$$Lambda$31/1875283985
> Serialization trace:
> factory (org.apache.beam.runners.core.metrics.MetricsMap)
> counters (org.apache.beam.runners.core.metrics.MetricsContainerImpl)
> metricsContainers
> (org.apache.beam.runners.core.metrics.MetricsContainerStepMap)
> metricsContainers
> (org.apache.beam.runners.spark.io.SparkUnboundedSource$Metadata)
> at
> 
> org.apache.beam.runners.spark.SparkPipelineResult.runtimeExceptionFrom(SparkPipelineResult.java:55)
> at
> 
> org.apache.beam.runners.spark.SparkPipelineResult.beamExceptionFrom(SparkPipelineResult.java:72)
> at
> 
> org.apache.beam.runners.spark.SparkPipelineResult.access$000(SparkPipelineResult.java:41)
> at
> 
> org.apache.beam.runners.spark.SparkPipelineResult$StreamingMode.stop(SparkPipelineResult.java:163)
> at
> 
> org.apache.beam.runners.spark.SparkPipelineResult.offerNewState(SparkPipelineResult.java:198)
> at
> 
> org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:101)
> at
> 
> org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:87)
> at
> 
> org.apache.beam.examples.BugWithKryoOnSpark.main(BugWithKryoOnSpark.java:75)
> 
> 
> I created a jira ticket and attached a project example on it,
> https://issues.apache.org/jira/browse/BEAM-4597
> 
> 
> Any feedback is appreciated.
> 
> -- 
> 
> JC 
> 
> 
> 
> 
> -- 
> 
> JC 
> 

-- 
Jean-Baptiste Onofré
jbono...@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com


Re: Beam SparkRunner and Spark KryoSerializer problem

2018-07-30 Thread Juan Carlos Garcia
Bump!

Does any of the core-dev roam around here?

Can someone provide a feedback about BEAM-4597


Thanks and regards,

On Thu, Jul 19, 2018 at 3:41 PM, Juan Carlos Garcia 
wrote:

> Folks,
>
> Its someone using the SparkRunner out there with the Spark KryoSerializer ?
>
> We are being force to use the not so efficient 'JavaSerializer' with Spark
> because we face the following exception:
>
> 
> Exception in thread "main" java.lang.RuntimeException: 
> org.apache.spark.SparkException:
> Job aborted due to stage failure: Exception while getting task result:
> com.esotericsoftware.kryo.KryoException: Unable to find class:
> org.apache.beam.runners.core.metrics.MetricsContainerImpl$$
> Lambda$31/1875283985
> Serialization trace:
> factory (org.apache.beam.runners.core.metrics.MetricsMap)
> counters (org.apache.beam.runners.core.metrics.MetricsContainerImpl)
> metricsContainers (org.apache.beam.runners.core.metrics.
> MetricsContainerStepMap)
> metricsContainers (org.apache.beam.runners.spark.io.SparkUnboundedSource$
> Metadata)
> at org.apache.beam.runners.spark.SparkPipelineResult.runtimeExceptionFrom(
> SparkPipelineResult.java:55)
> at org.apache.beam.runners.spark.SparkPipelineResult.beamExceptionFrom(
> SparkPipelineResult.java:72)
> at org.apache.beam.runners.spark.SparkPipelineResult.access$
> 000(SparkPipelineResult.java:41)
> at org.apache.beam.runners.spark.SparkPipelineResult$StreamingMode.stop(
> SparkPipelineResult.java:163)
> at org.apache.beam.runners.spark.SparkPipelineResult.offerNewState(
> SparkPipelineResult.java:198)
> at org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(
> SparkPipelineResult.java:101)
> at org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(
> SparkPipelineResult.java:87)
> at org.apache.beam.examples.BugWithKryoOnSpark.main(
> BugWithKryoOnSpark.java:75)
> 
>
> I created a jira ticket and attached a project example on it,
> https://issues.apache.org/jira/browse/BEAM-4597
>
> Any feedback is appreciated.
>
> --
>
> JC
>
>


-- 

JC


Beam SparkRunner and Spark KryoSerializer problem

2018-07-19 Thread Juan Carlos Garcia
Folks,

Its someone using the SparkRunner out there with the Spark KryoSerializer ?

We are being force to use the not so efficient 'JavaSerializer' with Spark
because we face the following exception:


Exception in thread "main" java.lang.RuntimeException:
org.apache.spark.SparkException: Job aborted due to stage failure:
Exception while getting task result:
com.esotericsoftware.kryo.KryoException: Unable to find class:
org.apache.beam.runners.core.metrics.MetricsContainerImpl$$Lambda$31/1875283985
Serialization trace:
factory (org.apache.beam.runners.core.metrics.MetricsMap)
counters (org.apache.beam.runners.core.metrics.MetricsContainerImpl)
metricsContainers
(org.apache.beam.runners.core.metrics.MetricsContainerStepMap)
metricsContainers
(org.apache.beam.runners.spark.io.SparkUnboundedSource$Metadata)
at
org.apache.beam.runners.spark.SparkPipelineResult.runtimeExceptionFrom(SparkPipelineResult.java:55)
at
org.apache.beam.runners.spark.SparkPipelineResult.beamExceptionFrom(SparkPipelineResult.java:72)
at
org.apache.beam.runners.spark.SparkPipelineResult.access$000(SparkPipelineResult.java:41)
at
org.apache.beam.runners.spark.SparkPipelineResult$StreamingMode.stop(SparkPipelineResult.java:163)
at
org.apache.beam.runners.spark.SparkPipelineResult.offerNewState(SparkPipelineResult.java:198)
at
org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:101)
at
org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:87)
at
org.apache.beam.examples.BugWithKryoOnSpark.main(BugWithKryoOnSpark.java:75)


I created a jira ticket and attached a project example on it,
https://issues.apache.org/jira/browse/BEAM-4597

Any feedback is appreciated.

-- 

JC