I went through the string conversions. Do you have an example of writing out XML/JSON/etc too?
On Tue, Nov 29, 2016 at 3:46 PM Jean-Baptiste Onofré <j...@nanthrax.net> wrote: > Hi Jesse, > > > https://github.com/jbonofre/incubator-beam/tree/DATAFORMAT/sdks/java/extensions/dataformat > > it's very simple and stupid and of course not complete at all (I have > other commits but not merged as they need some polishing), but as I > said, it's a base of discussion. > > Regards > JB > > On 11/29/2016 09:23 PM, Jesse Anderson wrote: > > @jb Sounds good. Just let us know once you've pushed. > > > > On Tue, Nov 29, 2016 at 2:54 PM Jean-Baptiste Onofré <j...@nanthrax.net> > > wrote: > > > >> Good point Eugene. > >> > >> Right now, it's a DoFn collection to experiment a bit (a pure > >> extension). It's pretty stupid ;) > >> > >> But, you are right, depending the direction of such extension, it could > >> cover more use cases (even if it's not my first intention ;)). > >> > >> Let me push the branch (pretty small) as an illustration, and in the > >> mean time, I'm preparing a document (more focused on the use cases). > >> > >> WDYT ? > >> > >> Regards > >> JB > >> > >> On 11/29/2016 08:47 PM, Eugene Kirpichov wrote: > >>> Hi JB, > >>> Depending on the scope of what you want to ultimately accomplish with > >> this > >>> extension, I think it may make sense to write a proposal document and > >>> discuss it. > >>> If it's just a collection of utility DoFn's for various well-defined > >>> source/target format pairs, then that's probably not needed, but if > it's > >>> anything more, then I think it is. > >>> That will help avoid a lot of churn if people propose reasonable > >>> significant changes. > >>> > >>> On Tue, Nov 29, 2016 at 11:15 AM Jean-Baptiste Onofré <j...@nanthrax.net > > > >>> wrote: > >>> > >>>> By the way Jesse, I gonna push my DATAFORMAT branch on my github and I > >>>> will post on the dev mailing list when done. > >>>> > >>>> Regards > >>>> JB > >>>> > >>>> On 11/29/2016 07:01 PM, Jesse Anderson wrote: > >>>>> I want to bring this thread back up since we've had time to think > about > >>>> it > >>>>> more and make a plan. > >>>>> > >>>>> I think a format-specific converter will be more time consuming task > >> than > >>>>> we originally thought. It'd have to be a writer that takes another > >> writer > >>>>> as a parameter. > >>>>> > >>>>> I think a string converter can be done as a simple transform. > >>>>> > >>>>> I think we should start with a simple string converter and plan for a > >>>>> format-specific writer. > >>>>> > >>>>> What are your thoughts? > >>>>> > >>>>> Thanks, > >>>>> > >>>>> Jesse > >>>>> > >>>>> On Thu, Nov 10, 2016 at 10:33 AM Jesse Anderson < > je...@smokinghand.com > >>> > >>>>> wrote: > >>>>> > >>>>> I was thinking about what the outputs would look like last night. I > >>>>> realized that more complex formats like JSON and XML may or may not > >>>> output > >>>>> the data in a valid format. > >>>>> > >>>>> Doing a direct conversion on unbounded collections would work just > >> fine. > >>>>> They're self-contained. For writing out bounded collections, that's > >> where > >>>>> we'll hit the issues. This changes the uber conversion transform > into a > >>>>> transform that needs to be a writer. > >>>>> > >>>>> If a transform executes a JSON conversion on a per element basis, > we'd > >>>> get > >>>>> this: > >>>>> { > >>>>> "key": "value" > >>>>> }, { > >>>>> "key": "value" > >>>>> }, > >>>>> > >>>>> That isn't valid JSON. > >>>>> > >>>>> The conversion transform would need to know do several things when > >>>> writing > >>>>> out a file. It would need to add brackets for an array. Now we have: > >>>>> [ > >>>>> { > >>>>> "key": "value" > >>>>> }, { > >>>>> "key": "value" > >>>>> }, > >>>>> ] > >>>>> > >>>>> We still don't have valid JSON. We have to remove the last comma or > >> have > >>>>> the uber transform start putting in the commas, except for the last > >>>> element. > >>>>> > >>>>> [ > >>>>> { > >>>>> "key": "value" > >>>>> }, { > >>>>> "key": "value" > >>>>> } > >>>>> ] > >>>>> > >>>>> Only by doing this do we have valid JSON. > >>>>> > >>>>> I'd argue we'd have a similar issue with XML. Some parsers require a > >> root > >>>>> element for everything. The uber transform would have to put the root > >>>>> element tags at the beginning and end of the file. > >>>>> > >>>>> On Wed, Nov 9, 2016 at 11:36 PM Manu Zhang <owenzhang1...@gmail.com> > >>>> wrote: > >>>>> > >>>>> I would love to see a lean core and abundant Transforms at the same > >> time. > >>>>> > >>>>> Maybe we can look at what Confluent <https://github.com/confluentinc > > > >>>> does > >>>>> for kafka-connect. They have official extensions support for JDBC, > HDFS > >>>> and > >>>>> ElasticSearch under https://github.com/confluentinc. They put them > >> along > >>>>> with other community extensions on > >>>>> https://www.confluent.io/product/connectors/ for visibility. > >>>>> > >>>>> Although not a commercial company, can we have a GitHub user like > >>>>> beam-community to host projects we build around beam but not suitable > >> for > >>>>> https://github.com/apache/incubator-beam. In the future, we may have > >>>>> beam-algebra like http://github.com/twitter/algebird for algebra > >>>> operations > >>>>> and beam-ml / beam-dl for machine learning / deep learning. Also, > there > >>>>> will will be beam related projects elsewhere maintained by other > >>>>> communities. We can put all of them on the beam-website or like spark > >>>>> packages as mentioned by Amit. > >>>>> > >>>>> My $0.02 > >>>>> Manu > >>>>> > >>>>> > >>>>> > >>>>> On Thu, Nov 10, 2016 at 2:59 AM Kenneth Knowles > <k...@google.com.invalid > >>> > >>>>> wrote: > >>>>> > >>>>>> On this point from Amit and Ismaël, I agree: we could benefit from a > >>>> place > >>>>>> for miscellaneous non-core helper transformations. > >>>>>> > >>>>>> We have sdks/java/extensions but it is organized as separate > >> artifacts. > >>>> I > >>>>>> think that is fine, considering the nature of Join and SortValues. > But > >>>> for > >>>>>> simpler transforms, Importing one artifact per tiny transform is too > >>>> much > >>>>>> overhead. It also seems unlikely that we will have enough > commonality > >>>>> among > >>>>>> the transforms to call the artifact anything other than [some > synonym > >>>> for] > >>>>>> "miscellaneous". > >>>>>> > >>>>>> I wouldn't want to take this too far - even though the SDK many > >>>>> transforms* > >>>>>> that are not required for the model [1], I like that the SDK > artifact > >>>> has > >>>>>> everything a user might need in their "getting started" phase of > use. > >>>> This > >>>>>> user-friendliness (the user doesn't care that ParDo is core and Sum > is > >>>>> not) > >>>>>> plus the difficulty of judging which transforms go where, are > probably > >>>> why > >>>>>> we have them mostly all in one place. > >>>>>> > >>>>>> Models to look at, off the top of my head, include Pig's PiggyBank > and > >>>>>> Apex's Malhar. These have different levels of support implied. > Others? > >>>>>> > >>>>>> Kenn > >>>>>> > >>>>>> [1] ApproximateQuantiles, ApproximateUnique, Count, Distinct, > Filter, > >>>>>> FlatMapElements, Keys, Latest, MapElements, Max, Mean, Min, Values, > >>>>> KvSwap, > >>>>>> Partition, Regex, Sample, Sum, Top, Values, WithKeys, WithTimestamps > >>>>>> > >>>>>> * at least they are separate classes and not methods on PCollection > >> :-) > >>>>>> > >>>>>> > >>>>>> On Wed, Nov 9, 2016 at 6:03 AM, Ismaël Mejía <ieme...@gmail.com> > >> wrote: > >>>>>> > >>>>>>> Nice discussion, and thanks Jesse for bringing this subject back. > >>>>>>> > >>>>>>> I agree 100% with Amit and the idea of having a home for those > >>>>> transforms > >>>>>>> that are not core enough to be part of the sdk, but that we all end > >> up > >>>>>>> re-writing somehow. > >>>>>>> > >>>>>>> This is a needed improvement to be more developer friendly, but > also > >> as > >>>>> a > >>>>>>> reference of good practices of Beam development, and for this > reason > >> I > >>>>>>> agree with JB that at this moment it would be better for these > >>>>> transforms > >>>>>>> to reside in the Beam repository at least for visibility reasons. > >>>>>>> > >>>>>>> One additional question is if these transforms represent a > different > >>>> DSL > >>>>>> or > >>>>>>> if those could be grouped with the current extensions (e.g. Join > and > >>>>>>> SortValues) into something more general that we as a community > could > >>>>>>> maintain, but well even if it is not the case, it would be really > >> nice > >>>>> to > >>>>>>> start working on something like this. > >>>>>>> > >>>>>>> Ismaël Mejía > >>>>>>> > >>>>>>> > >>>>>>> On Wed, Nov 9, 2016 at 11:59 AM, Jean-Baptiste Onofré < > >> j...@nanthrax.net > >>>>> > >>>>>>> wrote: > >>>>>>> > >>>>>>>> Related to spark-package, we also have Apache Bahir to host > >>>>>>>> connectors/transforms for Spark and Flink. > >>>>>>>> > >>>>>>>> IMHO, right now, Beam should host this, not sure if it makes sense > >>>>>>>> directly in the core. > >>>>>>>> > >>>>>>>> It reminds me the "Integration" DSL we discussed in the technical > >>>>>> vision > >>>>>>>> document. > >>>>>>>> > >>>>>>>> Regards > >>>>>>>> JB > >>>>>>>> > >>>>>>>> > >>>>>>>> On 11/09/2016 11:17 AM, Amit Sela wrote: > >>>>>>>> > >>>>>>>>> I think Jesse has a very good point on one hand, while Luke's and > >>>>>>>>> Kenneth's > >>>>>>>>> worries about committing users to specific implementations is in > >>>>>> place. > >>>>>>>>> > >>>>>>>>> The Spark community has a 3rd party repository for useful > libraries > >>>>>> that > >>>>>>>>> for various reasons are not a part of the Apache Spark project: > >>>>>>>>> https://spark-packages.org/. > >>>>>>>>> > >>>>>>>>> Maybe a "common-transformations" package would serve both users > >> quick > >>>>>>>>> ramp-up and ease-of-use while keeping Beam more "enabling" ? > >>>>>>>>> > >>>>>>>>> On Tue, Nov 8, 2016 at 9:03 PM Kenneth Knowles > >>>>> <k...@google.com.invalid > >>>>>>> > >>>>>>>>> wrote: > >>>>>>>>> > >>>>>>>>> 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 > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>> > >>>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>> > >>>>>>>>> > >>>>>>>> -- > >>>>>>>> Jean-Baptiste Onofré > >>>>>>>> jbono...@apache.org > >>>>>>>> http://blog.nanthrax.net > >>>>>>>> Talend - http://www.talend.com > >>>>>>>> > >>>>>>> > >>>>>> > >>>>> > >>>> > >>>> -- > >>>> Jean-Baptiste Onofré > >>>> jbono...@apache.org > >>>> http://blog.nanthrax.net > >>>> Talend - http://www.talend.com > >>>> > >>> > >> > >> -- > >> Jean-Baptiste Onofré > >> jbono...@apache.org > >> http://blog.nanthrax.net > >> Talend - http://www.talend.com > >> > > > > -- > Jean-Baptiste Onofré > jbono...@apache.org > http://blog.nanthrax.net > Talend - http://www.talend.com >