[FLINK-3854] Support Avro key-value rolling sink writer #1953

On 27 April 2016 at 19:29, Igor Berman <igor.ber...@gmail.com> wrote:

> Hi Aljoscha,
>
> avro-mapred jar contains different M/R output formats for avro, and their
> writers
> it's primary used in M/R jobs that produce avro output
> see some details here : https://avro.apache.org/docs/1.7.6/mr.html
>
> I have extracted(kind of copy-pasted+adjustments) some of the classes from
> there to remove this dependency and it's seems to work with basic scenario
> I still want to write it as if it was created with M/R job to be
> compatible with this library(e.g. key-value pairs are wrapped into
> AvroKeyValue object) so that it's not important if Flink or regular M/R
> created this files, still their consumer can read them in a same way
> WDYT?
>
>
>
>
>
> On 27 April 2016 at 11:27, Aljoscha Krettek <aljos...@apache.org> wrote:
>
>> Hi,
>> which code did you reuse from there? I asked Robert and I think it is
>> somewhat problematic to add these somewhat bigger dependencies.
>>
>> Cheers,
>> Aljoscha
>>
>> On Mon, 25 Apr 2016 at 21:24 Igor Berman <igor.ber...@gmail.com> wrote:
>>
>>> Hi,
>>> it's not a problem, I'll find time to change it(I understand the
>>> refactoring is in master and not released yet).
>>> Wanted to ask if it's acceptable to add following dependency to flink?
>>> I mean my code reused code in this jar(pay attention it's not present
>>> currently in flink classpath)
>>> <dependency>
>>> <groupId>org.apache.avro</groupId>
>>> <artifactId>avro-mapred</artifactId>
>>> <version>1.7.6</version>
>>> <classifier>hadoop2</classifier>
>>> </dependency>
>>>
>>> On 25 April 2016 at 16:20, Aljoscha Krettek <aljos...@apache.org> wrote:
>>>
>>>> Hi,
>>>> the code looks very good! Do you think it can be adapted to the
>>>> slightly modified interface introduced here:
>>>> https://issues.apache.org/jira/browse/FLINK-3637
>>>>
>>>> It basically requires the writer to know the write position, so that we
>>>> can truncate to a valid position in case of failure.
>>>>
>>>> Cheers,
>>>> Aljoscha
>>>>
>>>> On Thu, 21 Apr 2016 at 18:40 Igor Berman <igor.ber...@gmail.com> wrote:
>>>>
>>>>> ok,
>>>>> I have working prototype already, if somebody is interested(attached)
>>>>>
>>>>> I might add it as PR latter(with tests etc)
>>>>>
>>>>> tested locally & with s3
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On 21 April 2016 at 12:01, Aljoscha Krettek <aljos...@apache.org>
>>>>> wrote:
>>>>>
>>>>>> Hi,
>>>>>> as far as I know there is no one working on this. I'm only aware of
>>>>>> someone working on an ORC (from Hive) Writer.
>>>>>>
>>>>>> This would be a welcome addition! I think you are already on the
>>>>>> right track, the only thing required will probably be an AvroFileWriter 
>>>>>> and
>>>>>> you already started looking at SequenceFileWriter, which should be 
>>>>>> similar.
>>>>>>
>>>>>> Cheers,
>>>>>> Aljoscha
>>>>>>
>>>>>> On Thu, 21 Apr 2016 at 09:45 Igor Berman <igor.ber...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi All,
>>>>>>> Is there such implementation somewhere?(before I start to implement
>>>>>>> it myself, it seems not too difficult based on SequenceFileWriter 
>>>>>>> example)
>>>>>>>
>>>>>>> anyway any ideas/pointers will be highly appreciated
>>>>>>>
>>>>>>> thanks in advance
>>>>>>>
>>>>>>>
>>>>>
>>>
>

Reply via email to