It seems useful for small scale debugging / demoing to have
Dump.toString(). I think it should be named to clearly indicate its limited
scope. Maybe other stuff could go in the Dump namespace, but
"Dump.toJson()" would be for humans to read - so it should be pretty
printed, not treated as a machine-to-machine wire format.

The broader question of representing data in JSON or XML, etc, is already
the subject of many mature libraries which are already easy to use with
Beam.

The more esoteric practice of implicit or semi-implicit coercions seems
like it is also already addressed in many ways elsewhere.
Transform.via(TypeConverter) is basically the same as
MapElements.via(<lambda>) and also easy to use with Beam.

In both of the last cases, there are many reasonable approaches, and we
shouldn't commit our users to one of them.

On Tue, Nov 8, 2016 at 10:15 AM, Lukasz Cwik <lc...@google.com.invalid>
wrote:

> The suggestions you give seem good except for the the XML cases.
>
> Might want to have the XML be a document per line similar to the JSON
> examples you have been giving.
>
> On Tue, Nov 8, 2016 at 12:00 PM, Jesse Anderson <je...@smokinghand.com>
> wrote:
>
> > @lukasz Agreed there would have to be KV handling. I was more think that
> > whatever the addition, it shouldn't just handle KV. It should handle
> > Iterables, Lists, Sets, and KVs.
> >
> > For JSON and XML, I wonder if we'd be able to give someone something
> > general purpose enough that you would just end up writing your own code
> to
> > handle it anyway.
> >
> > Here are some ideas on what it could look like with a method and the
> > resulting string output:
> > *Stringify.toJSON()*
> >
> > With KV:
> > {"key": "value"}
> >
> > With Iterables:
> > ["one", "two", "three"]
> >
> > *Stringify.toXML("rootelement")*
> >
> > With KV:
> > <rootelement key=value />
> >
> > With Iterables:
> > <rootelement>
> >   <item>one</item>
> >   <item>two</item>
> >   <item>three</item>
> > </rootelement>
> >
> > *Stringify.toDelimited(",")*
> >
> > With KV:
> > key,value
> >
> > With Iterables:
> > one,two,three
> >
> > Do you think that would strike a good balance between reusable code and
> > writing your own for more difficult formatting?
> >
> > Thanks,
> >
> > Jesse
> >
> > On Tue, Nov 8, 2016 at 11:01 AM Lukasz Cwik <lc...@google.com.invalid>
> > wrote:
> >
> > Jesse, I believe if one format gets special treatment in TextIO, people
> > will then ask why doesn't JSON, XML, ... also not supported.
> >
> > Also, the example that you provide is using the fact that the input
> format
> > is an Iterable<Item>. You had posted a question about using KV with
> > TextIO.Write which wouldn't align with the proposed input format and
> still
> > would require to write a type conversion function, this time from KV to
> > Iterable<Item> instead of KV to string.
> >
> > On Tue, Nov 8, 2016 at 9:50 AM, Jesse Anderson <je...@smokinghand.com>
> > wrote:
> >
> > > Lukasz,
> > >
> > > I don't think you'd need complicated logic for TextIO.Write. For CSV
> the
> > > call would look like:
> > > Stringify.to("", ",", "\n");
> > >
> > > Where the arguments would be Stringify.to(prefix, delimiter, suffix).
> > >
> > > The code would be something like:
> > > StringBuffer buffer = new StringBuffer(prefix);
> > >
> > > for (Item item : list) {
> > >   buffer.append(item.toString());
> > >
> > >   if(notLast) {
> > >     buffer.append(delimiter);
> > >   }
> > > }
> > >
> > > buffer.append(suffix);
> > >
> > > c.output(buffer.toString());
> > >
> > > That would allow you to do the basic CSV, TSV, and other formats
> without
> > > complicated logic. The same sort of thing could be done for
> TextIO.Write.
> > >
> > > Thanks,
> > >
> > > Jesse
> > >
> > > On Tue, Nov 8, 2016 at 10:30 AM Lukasz Cwik <lc...@google.com.invalid>
> > > wrote:
> > >
> > > > The conversion from object to string will have uses outside of just
> > > > TextIO.Write so it seems logical that we would want to have a ParDo
> do
> > > the
> > > > conversion.
> > > >
> > > > Text file formats have a lot of variance, even if you consider the
> > subset
> > > > of CSV like formats where it could have fixed width fields, or
> escaping
> > > and
> > > > quoting around other fields, or headers that should be placed at the
> > top.
> > > >
> > > > Having all these format conversions within TextIO.Write seems like a
> > lot
> > > of
> > > > logic to contain in that transform which should just focus on writing
> > to
> > > > files.
> > > >
> > > > On Tue, Nov 8, 2016 at 8:15 AM, Jesse Anderson <
> je...@smokinghand.com>
> > > > wrote:
> > > >
> > > > > This is a thread moved over from the user mailing list.
> > > > >
> > > > > I think there needs to be a way to convert a PCollection<KV> to
> > > > > PCollection<String> Conversion.
> > > > >
> > > > > To do a minimal WordCount, you have to manually convert the KV to a
> > > > String:
> > > > >         p
> > > > >                 .apply(TextIO.Read.from("playing_cards.tsv"))
> > > > >                 .apply(Regex.split("\\W+"))
> > > > >                 .apply(Count.perElement())
> > > > > *                .apply(MapElements.via((KV<String, Long> count)
> ->*
> > > > > *                            count.getKey() + ":" +
> count.getValue()*
> > > > > *                        ).withOutputType(
> > TypeDescriptors.strings()))*
> > > > >                 .apply(TextIO.Write.to("output/stringcounts"));
> > > > >
> > > > > This code really should be something like:
> > > > >         p
> > > > >                 .apply(TextIO.Read.from("playing_cards.tsv"))
> > > > >                 .apply(Regex.split("\\W+"))
> > > > >                 .apply(Count.perElement())
> > > > > *                .apply(ToString.stringify())*
> > > > >                 .apply(TextIO.Write.to("output/stringcounts"));
> > > > >
> > > > > To summarize the discussion:
> > > > >
> > > > >    - JA: Add a method to StringDelegateCoder to output any KV or
> list
> > > > >    - JA and DH: Add a SimpleFunction that takes an type and runs
> > > > toString()
> > > > >    on it:
> > > > >    class ToStringFn<InputT> extends SimpleFunction<InputT, String>
> {
> > > > >        public static String apply(InputT input) {
> > > > >            return input.toString();
> > > > >        }
> > > > >    }
> > > > >    - JB: Add a general purpose type converter like in Apache Camel.
> > > > >    - JA: Add Object support to TextIO.Write that would write out
> the
> > > > >    toString of any Object.
> > > > >
> > > > > My thoughts:
> > > > >
> > > > > Is converting to a PCollection<String> mostly needed when you're
> > using
> > > > > TextIO.Write? Will a general purpose transform only work in certain
> > > cases
> > > > > and you'll normally have to write custom code format the strings
> the
> > > way
> > > > > you want them?
> > > > >
> > > > > IMHO, it's yes to both. I'd prefer to add Object support to
> > > TextIO.Write
> > > > or
> > > > > a SimpleFunction that takes a delimiter as an argument. Making a
> > > > > SimpleFunction that's able to specify a delimiter (and perhaps a
> > prefix
> > > > and
> > > > > suffix) should cover the majority of formats and cases.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jesse
> > > > >
> > > >
> > >
> >
>

Reply via email to