Hi all,
After some great discussion, I think we have at least reached a consensus
that we can have a unified sink to handle streaming, batch, hive and HDFS.
And for FileSystem connector, undoubtedly, we reuse DataStream
StreamingFileSink.
I updated the FLIP:
1.Move external HDFS and close exactl
Hi,
I am very interested in the topic. I would like to join the offline
discussion if possible. I think you guys already give many inputs and
concerns. I would share some of my thoughts. Correct me if I misunderstand
you.
Flink is a unified engine. Since that Flink should provide the e2e
exactly
Hi Stephan & Kostas & Piotrek, thanks for these inputs,
Maybe what I expressed is not clear. For the implementation, I want to know
what you think, rather than must making another set from scratch. Piotrek
you are right, implementation is the part of this FLIP too, because we can
not list all deta
Hi Kurt,
+1 for having some offline discussion on this topic.
But I think the question about using StreamingFileSink or implementing subset
of it’s feature from scratch is quite fundamental design decision, with impact
on the behaviour of Public API, so I wouldn’t discard it as technical detail
Hi all,
Thanks for the discuss and feedbacks. I think this FLIP doesn't imply the
implementation
of such connector yet, it only describes the functionality and expected
behaviors from user's
perspective. Reusing current StreamingFileSink is definitely one of the
possible ways to
implement it. Sinc
Hi all,
I also agree with Stephan on this!
It has been more than a year now that most of our efforts have had the
"unify" / "unification"/ etc either on their title or in their core
and this has been the focus of all our resources. By deviating from
this now, we only put more stress on other team
>> The FLIP is "Filesystem connector in Table", it's about building up
Flink Table's capabilities.
That is exactly what worries me. The whole effort is not thinking about
Flink as a whole any more.
This proposal is not trying to build a consistent user experience across
batch and streaming, across
Hi Stephan, Thanks very much for your detailed reply.
*## StreamingFileSink not support writer with path*
The FLIP is "Filesystem connector in Table", it's about building up Flink
Table's capabilities. But I think Hive is important, I see that most users
use Flink and Spark to write data from Kaf
Hi Jingsong ,
I am looking forward this feature. Because in some streaming application,it
need transfer their messages to hdfs , in order to offline analysis.
Best wishes,
LakeShen
Stephan Ewen 于2020年3月17日周二 下午7:42写道:
> I would really like to see us converging the stack and the functionality
>
I would really like to see us converging the stack and the functionality
here.
Meaning to try and use the same sinks in the Table API as for the
DataStream API, and using the same sink for batch and streaming.
The StreamingFileSink has a lot of things that can help with that. If
possible, it would
Hi Piotr,
I am very entangled.
Let me re-list the table streaming sink requirements:
- In table, maybe 90% sinks are for Hive. The parquet and orc are the most
important formats. Hive provide RecordWriters, it is easy to support all
hive formats by using it, and we don't need concern hive version
Hi Jingsong,
> First way is reusing Batch sink in FLINK-14254, It has handled the partition
> and metastore logic well.
> - unify batch and streaming
> - Using FileOutputFormat is consistent with FileInputFormat.
> - Add exactly-once related logic. Just 200+ lines code.
> - It's natural to suppo
Thanks Jinhai for involving.
> we need add 'connector.sink.username' for UserGroupInformation when data
is written to HDFS
Yes, I am not an expert of HDFS, but it seems we need do this "doAs" in the
code for access external HDFS. I will update document.
Best,
Jingsong Lee
On Mon, Mar 16, 2020 a
Thanks Piotr and Yun for involving.
Hi Piotr and Yun, for implementation,
FLINK-14254 [1] introduce batch sink table world, it deals with partitions
thing, metastore thing and etc.. And it just reuse Dataset/Datastream
FileInputFormat and FileOutputFormat. Filesystem can not do without
FileInputF
Hi,
Very thanks for Jinsong to bring up this discussion! It should largely
improve the usability after enhancing the FileSystem connector in Table.
I have the same question with Piotr. From my side, I think it should be
better to be able to reuse existing StreamingFileSink.
Hi,
Which actual sinks/sources are you planning to use in this feature? Is it about
exposing StreamingFileSink in the Table API? Or do you want to implement new
Sinks/Sources?
Piotrek
> On 13 Mar 2020, at 10:04, jinhai wang wrote:
>
> Thanks for FLIP-115. It is really useful feature for plat
Thanks for FLIP-115. It is really useful feature for platform developers who
manage hundreds of Flink to Hive jobs in production.
I think we need add 'connector.sink.username' for UserGroupInformation when
data is written to HDFS
在 2020/3/13 下午3:33,“Jingsong Li” 写入:
Hi everyone,
Thanks for FLIP-115. It is really useful feature for platform developers
who manage hundreds of Flink to Hive jobs in production.
I think we need add 'connector.sink.username' for UserGroupInformation when
data is written to HDFS
Jingsong Li 于2020年3月13日周五 下午3:33写道:
> Hi everyone,
>
> I'd like to
18 matches
Mail list logo