Actually, I have been an idea, how about support hive on flink ? Since lots of business are written by hive sql. And users wants to transform map reduce to fink without changing the sql.
Zhangminglei > 在 2018年6月17日,下午8:11,zhangminglei <18717838...@163.com> 写道: > > Hi, Sagar > > There already has relative JIRAs for ORC and Parquet, you can take a look > here: > > https://issues.apache.org/jira/browse/FLINK-9407 > <https://issues.apache.org/jira/browse/FLINK-9407> and > https://issues.apache.org/jira/browse/FLINK-9411 > <https://issues.apache.org/jira/browse/FLINK-9411> > > For ORC format, Currently only support basic data types, such as Long, > Boolean, Short, Integer, Float, Double, String. > > Best > Zhangminglei > > > >> 在 2018年6月17日,上午11:11,sagar loke <sagar...@gmail.com> 写道: >> >> We are eagerly waiting for >> >> - Extends Streaming Sinks: >> - Bucketing Sink should support S3 properly (compensate for eventual >> consistency), work with Flink's shaded S3 file systems, and efficiently >> support formats that compress/index arcoss individual rows (Parquet, ORC, >> ...) >> >> Especially for ORC and Parquet sinks. Since, We are planning to use >> Kafka-jdbc to move data from rdbms to hdfs. >> >> Thanks, >> >> On Sat, Jun 16, 2018 at 5:08 PM Elias Levy <fearsome.lucid...@gmail.com >> <mailto:fearsome.lucid...@gmail.com>> wrote: >> One more, since it we have to deal with it often: >> >> - Idling sources (Kafka in particular) and proper watermark propagation: >> FLINK-5018 / FLINK-5479 >> >> On Fri, Jun 8, 2018 at 2:58 PM, Elias Levy <fearsome.lucid...@gmail.com >> <mailto:fearsome.lucid...@gmail.com>> wrote: >> Since wishes are free: >> >> - Standalone cluster job isolation: >> https://issues.apache.org/jira/browse/FLINK-8886 >> <https://issues.apache.org/jira/browse/FLINK-8886> >> - Proper sliding window joins (not overlapping hoping window joins): >> https://issues.apache.org/jira/browse/FLINK-6243 >> <https://issues.apache.org/jira/browse/FLINK-6243> >> - Sharing state across operators: >> https://issues.apache.org/jira/browse/FLINK-6239 >> <https://issues.apache.org/jira/browse/FLINK-6239> >> - Synchronizing streams: https://issues.apache.org/jira/browse/FLINK-4558 >> <https://issues.apache.org/jira/browse/FLINK-4558> >> >> Seconded: >> - Atomic cancel-with-savepoint: >> https://issues.apache.org/jira/browse/FLINK-7634 >> <https://issues.apache.org/jira/browse/FLINK-7634> >> - Support dynamically changing CEP patterns : >> https://issues.apache.org/jira/browse/FLINK-7129 >> <https://issues.apache.org/jira/browse/FLINK-7129> >> >> >> On Fri, Jun 8, 2018 at 1:31 PM, Stephan Ewen <se...@apache.org >> <mailto:se...@apache.org>> wrote: >> Hi all! >> >> Thanks for the discussion and good input. Many suggestions fit well with the >> proposal above. >> >> Please bear in mind that with a time-based release model, we would release >> whatever is mature by end of July. >> The good thing is we could schedule the next release not too far after that, >> so that the features that did not quite make it will not be delayed too long. >> In some sense, you could read this as as "what to do first" list, rather >> than "this goes in, other things stay out". >> >> Some thoughts on some of the suggestions >> >> Kubernetes integration: An opaque integration with Kubernetes should be >> supported through the "as a library" mode. For a deeper integration, I know >> that some committers have experimented with some PoC code. I would let Till >> add some thoughts, he has worked the most on the deployment parts recently. >> >> Per partition watermarks with idleness: Good point, could one implement that >> on the current interface, with a periodic watermark extractor? >> >> Atomic cancel-with-savepoint: Agreed, this is important. Making this work >> with all sources needs a bit more work. We should have this in the roadmap. >> >> Elastic Bloomfilters: This seems like an interesting new feature - the above >> suggested feature set was more about addressing some longer standing >> issues/requests. However, nothing should prevent contributors to work on >> that. >> >> Best, >> Stephan >> >> >> On Wed, Jun 6, 2018 at 6:23 AM, Yan Zhou [FDS Science] <yz...@coupang.com >> <mailto:yz...@coupang.com>> wrote: >> +1 on https://issues.apache.org/jira/browse/FLINK-5479 >> <https://issues.apache.org/jira/browse/FLINK-5479> >> [FLINK-5479] Per-partition watermarks in ... >> <https://issues.apache.org/jira/browse/FLINK-5479> >> issues.apache.org <http://issues.apache.org/> >> Reported in ML: >> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Kafka-topic-partition-skewness-causes-watermark-not-being-emitted-td11008.html >> >> <http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Kafka-topic-partition-skewness-causes-watermark-not-being-emitted-td11008.html> >> It's normally not a common case to have Kafka partitions not producing any >> data, but it'll probably be good to handle this as well. I ... >> >> From: Rico Bergmann <i...@ricobergmann.de <mailto:i...@ricobergmann.de>> >> Sent: Tuesday, June 5, 2018 9:12:00 PM >> To: Hao Sun >> Cc: dev@flink.apache.org <mailto:dev@flink.apache.org>; user >> Subject: Re: [DISCUSS] Flink 1.6 features >> >> +1 on K8s integration >> >> >> >> Am 06.06.2018 um 00:01 schrieb Hao Sun <ha...@zendesk.com >> <mailto:ha...@zendesk.com>>: >> >>> adding my vote to K8S Job mode, maybe it is this? >>>> Smoothen the integration in Container environment, like "Flink as a >>>> Library", and easier integration with Kubernetes services and other >>>> proxies. >>> >>> >>> >>> On Mon, Jun 4, 2018 at 11:01 PM Ben Yan <yan.xiao.bin.m...@gmail.com >>> <mailto:yan.xiao.bin.m...@gmail.com>> wrote: >>> Hi Stephan, >>> >>> Will [ https://issues.apache.org/jira/browse/FLINK-5479 >>> <https://issues.apache.org/jira/browse/FLINK-5479> ] (Per-partition >>> watermarks in FlinkKafkaConsumer should consider idle partitions) be >>> included in 1.6? As we are seeing more users with this issue on the mailing >>> lists. >>> >>> Thanks. >>> Ben >>> >>> 2018-06-05 5:29 GMT+08:00 Che Lui Shum <sh...@us.ibm.com >>> <mailto:sh...@us.ibm.com>>: >>> Hi Stephan, >>> >>> Will FLINK-7129 (Support dynamically changing CEP patterns) be included in >>> 1.6? There were discussions about possibly including it in 1.6: >>> http://mail-archives.apache.org/mod_mbox/flink-user/201803.mbox/%3cCAMq=ou7gru2o9jtowxn1lc1f7nkcxayn6a3e58kxctb4b50...@mail.gmail.com%3e >>> >>> <http://mail-archives.apache.org/mod_mbox/flink-user/201803.mbox/%3cCAMq=ou7gru2o9jtowxn1lc1f7nkcxayn6a3e58kxctb4b50...@mail.gmail.com%3e> >>> >>> Thanks, >>> Shirley Shum >>> >>> Stephan Ewen ---06/04/2018 02:21:47 AM---Hi Flink Community! The release of >>> Apache Flink 1.5 has happened (yay!) - so it is a good time >>> >>> From: Stephan Ewen <se...@apache.org <mailto:se...@apache.org>> >>> To: dev@flink.apache.org <mailto:dev@flink.apache.org>, user >>> <u...@flink.apache.org <mailto:u...@flink.apache.org>> >>> Date: 06/04/2018 02:21 AM >>> Subject: [DISCUSS] Flink 1.6 features >>> >>> >>> >>> Hi Flink Community! >>> >>> The release of Apache Flink 1.5 has happened (yay!) - so it is a good time >>> to start talking about what to do for release 1.6. >>> >>> == Suggested release timeline == >>> >>> I would propose to release around end of July (that is 8-9 weeks from now). >>> >>> The rational behind that: There was a lot of effort in release testing >>> automation (end-to-end tests, scripted stress tests) as part of release >>> 1.5. You may have noticed the big set of new modules under >>> "flink-end-to-end-tests" in the Flink repository. It delayed the 1.5 >>> release a bit, and needs to continue as part of the coming release cycle, >>> but should help make releasing more lightweight from now on. >>> >>> (Side note: There are also some nightly stress tests that we created and >>> run at data Artisans, and where we are looking whether and in which way it >>> would make sense to contribute them to Flink.) >>> >>> == Features and focus areas == >>> >>> We had a lot of big and heavy features in Flink 1.5, with FLIP-6, the new >>> network stack, recovery, SQL joins and client, ... Following something like >>> a "tick-tock-model", I would suggest to focus the next release more on >>> integrations, tooling, and reducing user friction. >>> >>> Of course, this does not mean that no other pull request gets reviewed, an >>> no other topic will be examined - it is simply meant as a help to >>> understand where to expect more activity during the next release cycle. >>> Note that these are really the coarse focus areas - don't read this as a >>> comprehensive list. >>> >>> This list is my first suggestion, based on discussions with committers, >>> users, and mailing list questions. >>> >>> - Support Java 9 and Scala 2.12 >>> >>> - Smoothen the integration in Container environment, like "Flink as a >>> Library", and easier integration with Kubernetes services and other proxies. >>> >>> - Polish the remaing parts of the FLIP-6 rewrite >>> >>> - Improve state backends with asynchronous timer snapshots, efficient >>> timer deletes, state TTL, and broadcast state support in RocksDB. >>> >>> - Extends Streaming Sinks: >>> - Bucketing Sink should support S3 properly (compensate for eventual >>> consistency), work with Flink's shaded S3 file systems, and efficiently >>> support formats that compress/index arcoss individual rows (Parquet, ORC, >>> ...) >>> - Support ElasticSearch's new REST API >>> >>> - Smoothen State Evolution to support type conversion on snapshot restore >>> >>> - Enhance Stream SQL and CEP >>> - Add support for "update by key" Table Sources >>> - Add more table sources and sinks (Kafka, Kinesis, Files, K/V stores) >>> - Expand SQL client >>> - Integrate CEP and SQL, through MATCH_RECOGNIZE clause >>> - Improve CEP Performance of SharedBuffer on RocksDB >>> >>> >>> >>> >>> >> >> >> >> -- >> Cheers, >> Sagar >