Hello Dan,

My implementation uses a normal DoFn, I am not sure if this case (the Log
one)
deserves to be a core primitive (and I understand that it is important to
keep
the number of core primitives small), however knowing how the PCollection
changes after one transform seems to me a common case, at least when you are
developing a Pipeline, I also agree that this is a problem that has many
possible solutions as Davor mentions, but I wanted to explore a generic
transform because I think it is the most appropriate solution (and because
changing the runner semantics has stronger consequences, and it is scary).

Another option is to consider the set of transforms that could be general
enough
to be an 'extra' library, but not part of the core ones, I imagine that
when I
start working with JB in the integration DSL some ideas will emerge, but I
can
imagine that you guys at google should have already found some ideas too.

Thanks for the link to the PR.

-Ismaël



On Sun, Mar 20, 2016 at 6:55 PM, Dan Halperin <[email protected]> wrote:

> Sorry for short phone email. Check out this Transform from a pull request.
>
>
> https://github.com/elibixby/DataflowJavaSDK/blob/51ee732964b3425c6c4a8677d135c41765d9bcdc/contrib/firebaseio/src/main/java/contrib/LogElements.java
>
> The key subtleties are around making sure you don't log too often--logging
> per element is very expensive.
>
> Happy to discuss more if you have questions, and again sorry I'm on a
> phone so can't provide a more comprehensive response.
>
> On Sun, Mar 20, 2016 at 10:21 Dan Halperin <[email protected]> wrote:
>
>> Hi Ismael,
>>
>> I think you can just do this with a normal DoFn. Why do you think this
>> needs to be a new primitive?
>> On Sun, Mar 20, 2016 at 10:20 Dan Halperin <[email protected]> wrote:
>>
>>> Hi is
>>> On Sun, Mar 20, 2016 at 08:18 Jean-Baptiste Onofré <[email protected]>
>>> wrote:
>>>
>>>> Hi,
>>>>
>>>> thanks for the update.
>>>>
>>>> IMHO, I would name Debug transform as Log:
>>>>
>>>> .apply(Log.withLevel("DEBUG"))
>>>> .apply(Log.withLevel("INFO").withPattern("%d %m ..."))
>>>> .apply(Log.withLevel("WARN").withMessage("Foo").withStream("System.out")
>>>>
>>>> It would more flexible and related to the actual behavior.
>>>>
>>>> I would mimic a bit the Camel log component for instance.
>>>>
>>>> If you don't mind, I will do it with you.
>>>>
>>>> Thanks
>>>> Regards
>>>> JB
>>>>
>>>> On 03/20/2016 12:07 PM, Ismaël Mejía wrote:
>>>> > Hi,
>>>> >
>>>> > The code of the transform is here in a playground for Beam
>>>> experiments I
>>>> > created (it is a bit alpha for the moment, and it does not have
>>>> comments):
>>>> >
>>>> >
>>>> https://github.com/iemejia/beam-playground/blob/master/src/main/java/org/apache/beam/transforms/Debug.java
>>>> >
>>>> > Since my initial goal was more of a test scenario in the
>>>> > DirectPipelineRunner I haven't considered yet more advanced logging
>>>> > capabilities and the possible issues of distribution (serialization,
>>>> in
>>>> > particular of dependencies, as well as exceptions, etc), but of course
>>>> > it is something I expect to improve if there is interest. Do you see
>>>> > some immediate things to improve to try it with the distributed
>>>> runners
>>>> > (I want to do this, as a excuse also to  try the FlinkRunner).
>>>> >
>>>> > Best,
>>>> > -Ismael
>>>> >
>>>> >
>>>> > On Sun, Mar 20, 2016 at 11:13 AM, Jean-Baptiste Onofré <
>>>> [email protected]
>>>> > <mailto:[email protected]>> wrote:
>>>> >
>>>> >     By the way, for the "Integration" DSL, in addition of explicit
>>>> debug
>>>> >     transform, it would make sense to have an implicit "Tracer". It's
>>>> >     something that I planned: it would allow us to have sampling on
>>>> >     PCollection if the pipeline tracer is enabled (like we do in a
>>>> Camel
>>>> >     route with the tracer).
>>>> >
>>>> >     Regards
>>>> >     JB
>>>> >
>>>> >     On 03/20/2016 10:14 AM, Ismaël Mejía wrote:
>>>> >
>>>> >         ​Hello,
>>>> >
>>>> >         I just started playing with Beam and I wanted to debug what
>>>> happens
>>>> >         between transforms in pipelines. I wrote a simple 'Debug'
>>>> >         transform for
>>>> >         this.
>>>> >         The idea is to apply a function based on a predicate to any
>>>> >         element in a
>>>> >         collection without changing the collection, or in other
>>>> words, a
>>>> >         transform that
>>>> >         does not transform but produces side effects.
>>>> >
>>>> >         The idea is better illustrated with this simple example:
>>>> >
>>>> >               .apply(FlatMapElements.via((String text) ->
>>>> >         Arrays.asList(text.split(" ")))
>>>> >                 .withOutputType(new TypeDescriptor<String>() {
>>>> >                }))
>>>> >               .apply(Debug
>>>> >                 .when((String s) -> s.startsWith("A"))
>>>> >                 .with((String s) -> {
>>>> >                   System.out.println(s);
>>>> >                   return null;
>>>> >                 }));
>>>> >               .apply(Filter.byPredicate((String text) ->
>>>> text.length() > 5))
>>>> >               .apply(Debug.print());  // sugared method, same as above
>>>> >
>>>> >         I think this can be useful (at least for debugging purposes),
>>>> is
>>>> >         there
>>>> >         something
>>>> >         like this already in the SDK ? If this is not the case, can
>>>> you
>>>> >         please
>>>> >         give me some
>>>> >         feedback/ideas to improve my transform.
>>>> >
>>>> >         Thanks,
>>>> >         -Ismael
>>>> >
>>>> >         ps. You can find the code of the first version of the
>>>> transform
>>>> >         here:
>>>> >
>>>> https://github.com/iemejia/beam-playground/blob/master/src/main/java/org/apache/beam/transforms/Debug.java
>>>> >
>>>> >
>>>> >
>>>> >     --
>>>> >     Jean-Baptiste Onofré
>>>> >     [email protected] <mailto:[email protected]>
>>>> >     http://blog.nanthrax.net
>>>> >     Talend - http://www.talend.com
>>>> >
>>>> >
>>>>
>>>> --
>>>> Jean-Baptiste Onofré
>>>> [email protected]
>>>> http://blog.nanthrax.net
>>>> Talend - http://www.talend.com
>>>>
>>>

Reply via email to