Re: [PROPOSAL] Introduce Elastic Bloom Filter For Flink

2018-05-23 Thread Elias Levy
I would suggest you consider an alternative data structures: a Cuckoo Filter or a Golumb Compressed Sequence. The GCS data structure was introduced in Cache-, Hash- and Space-Efficient Bloom Filters by F. Putze, P. Sanders,

Re: [DISCUSS] Flink 1.6 features

2018-06-08 Thread Elias Levy
Since wishes are free: - Standalone cluster job isolation: 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 - Sharing state across operators:

Re: [VOTE] Release 1.6.0, release candidate #2

2018-08-03 Thread Elias Levy
On Fri, Aug 3, 2018 at 9:23 AM Till Rohrmann wrote: > The complete staging area is available for your review, which includes: > * JIRA release notes [1], > [1] > https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12342760 Shouldn't the release notes only include issues

Re: [Discuss] Outer join support and timestamp assignment for IntervalJoin

2018-08-13 Thread Elias Levy
As a developer, while not quite a succinct, I feel that option A in both cases is easier to read. On Mon, Aug 13, 2018 at 4:18 AM Florian Schmidt wrote: > Hello Community, > > I’ve recently been working on adding support for outer joins [1] and > timestamp assignment [2] to the IntervalJoin in

Re: [DISCUSS] Release Flink 1.5.3

2018-08-16 Thread Elias Levy
If I may, think FLINK-10011 should be considered a blocker and included in 1.5.3 and a 1.4.3 release. On Thu, Aug 16, 2018 at 3:09 AM Dominik Wosiński wrote: > +1, I agree that frequent releases are good for users. > Dominik > > > Wysłane z aplikacji Poczta dla Windows 10 > > Od: Stefan Richter

Re: Flink 1.7 Development Priorities

2018-08-23 Thread Elias Levy
I would be nice to see the state TTL work finished. At the moment state will not expire if a key is not accessed while the job is running, which means jobs that require state expiration must continue using timers to expire data. On Thu, Aug 23, 2018 at 1:12 AM Aljoscha Krettek wrote: > Hi

Re: [DISCUSS] Breaking the Scala API for Scala 2.12 Support

2018-10-04 Thread Elias Levy
The second alternative, with the addition of methods that take functions with Scala types, seems the most sensible. I wonder if there is a need then to maintain the *J Java parameter methods, or whether users could just access the functionality by converting the Scala DataStreams to Java via

Re: Sharing state between subtasks

2018-10-08 Thread Elias Levy
Kafka Streams handles this problem, time alignment, by processing records from the partitions with the lowest timestamp in a best effort basis. See KIP-353 for the details. The same could be done within the Kafka source and multiple input stream operators. I opened FLINK-4558

Re: Sharing state between subtasks

2018-10-10 Thread Elias Levy
On Wed, Oct 10, 2018 at 8:15 AM Aljoscha Krettek wrote: > I think the two things (shared state and new source interface) are > somewhat orthogonal. The new source interface itself alone doesn't solve > the problem, we would still need some mechanism for sharing the event-time > information

Re: Sharing state between subtasks

2018-10-10 Thread Elias Levy
On Wed, Oct 10, 2018 at 9:33 AM Fabian Hueske wrote: > I think the new source interface would be designed to be able to leverage > shared state to achieve time alignment. > I don't think this would be possible without some kind of shared state. > > The problem of tasks that are far ahead in time

Re: Sharing state between subtasks

2018-10-09 Thread Elias Levy
On Tue, Oct 9, 2018 at 1:25 AM Aljoscha Krettek wrote: > @Elias Do you know if Kafka Consumers do this alignment across multiple > consumers or only within one Consumer across the partitions that it reads > from. > The behavior is part of Kafka Streams

Re: [DISCUSS] Release Flink 1.5.4 and 1.6.1

2018-09-14 Thread Elias Levy
Any chance we may see a 1.4.3 release with the ZK fixes? On Fri, Sep 14, 2018 at 8:29 AM Till Rohrmann wrote: > Hi everyone, > > it has already been a couple of weeks since we released Flink 1.5.3 and > Flink 1.6.0. In both release branches are important bug fixes from which > our users can

Re: [DISCUSS] Release Flink 1.5.4 and 1.6.1

2018-09-17 Thread Elias Levy
cause in 1.4 we didn't > have many automated e2e tests. Consequently, the release process would > require a considerable community effort for testing in order to make sure > that the release is properly working. > > What do you think? > > Cheers, > Till > > On Fri, Sep 14, 2

Re: [Discuss] Semantics of event time for state TTL

2019-04-04 Thread Elias Levy
My 2c: Timestamp stored with the state value: Event timestamp Timestamp used to check expiration: Last emitted watermark That follows the event time processing model used elsewhere is Flink. E.g. events are segregated into windows based on their event time, but the windows do not fire until the

Re: [DISCUSS] FLIP-33: Terminate/Suspend Job with Savepoint

2019-03-05 Thread Elias Levy
Apologies for the late reply. I think this is badly needed, but I fear we are adding complexity by introducing yet two more stop commands. We'll have: cancel, stop, terminate. and suspend. We basically want to do two things: terminate a job with prejudice or stop a job safely. For the former

Re: [DISCUSS] Improve Queryable State and introduce a QueryServerProxy component

2019-04-29 Thread Elias Levy
On Fri, Apr 26, 2019 at 8:58 PM vino yang wrote: > I agree with your opinion that "*Flink jobs don't sufficiently meet these > requirements to work as a replacement for a data store.*". Actually, I > think it's obviously not Flink's goal. > I would not be so sure. When data Artisans

Re: [DISCUSS] Improve Queryable State and introduce a QueryServerProxy component

2019-04-26 Thread Elias Levy
On Fri, Apr 26, 2019 at 1:41 AM vino yang wrote: > You are right, currently, the queryable state has few users. And I totally > agree with you, it makes the streaming works more like a DB. > Alas, I don't think queryable state will really be used much in production other than for ad hoc queries

[jira] [Created] (FLINK-4050) FlinkKafkaProducer API Refactor

2016-06-09 Thread Elias Levy (JIRA)
Elias Levy created FLINK-4050: - Summary: FlinkKafkaProducer API Refactor Key: FLINK-4050 URL: https://issues.apache.org/jira/browse/FLINK-4050 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-4027) FlinkKafkaProducer09 sink can lose messages

2016-06-06 Thread Elias Levy (JIRA)
Elias Levy created FLINK-4027: - Summary: FlinkKafkaProducer09 sink can lose messages Key: FLINK-4027 URL: https://issues.apache.org/jira/browse/FLINK-4027 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-3984) Event time of stream transformations is undocumented

2016-05-27 Thread Elias Levy (JIRA)
Elias Levy created FLINK-3984: - Summary: Event time of stream transformations is undocumented Key: FLINK-3984 URL: https://issues.apache.org/jira/browse/FLINK-3984 Project: Flink Issue Type

[jira] [Created] (FLINK-3710) ScalaDocs for org.apache.flink.streaming.scala are missing from the web site

2016-04-06 Thread Elias Levy (JIRA)
Elias Levy created FLINK-3710: - Summary: ScalaDocs for org.apache.flink.streaming.scala are missing from the web site Key: FLINK-3710 URL: https://issues.apache.org/jira/browse/FLINK-3710 Project: Flink

[jira] [Created] (FLINK-3692) Develop a Kafka state backend

2016-04-03 Thread Elias Levy (JIRA)
Elias Levy created FLINK-3692: - Summary: Develop a Kafka state backend Key: FLINK-3692 URL: https://issues.apache.org/jira/browse/FLINK-3692 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-4326) Flink start-up scripts should optionally start services on the foreground

2016-08-05 Thread Elias Levy (JIRA)
Elias Levy created FLINK-4326: - Summary: Flink start-up scripts should optionally start services on the foreground Key: FLINK-4326 URL: https://issues.apache.org/jira/browse/FLINK-4326 Project: Flink

[jira] [Created] (FLINK-4558) Add support for synchronizing streams

2016-09-01 Thread Elias Levy (JIRA)
Elias Levy created FLINK-4558: - Summary: Add support for synchronizing streams Key: FLINK-4558 URL: https://issues.apache.org/jira/browse/FLINK-4558 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-4497) Add support for Scala tuples and case classes to Cassandra sink

2016-08-25 Thread Elias Levy (JIRA)
Elias Levy created FLINK-4497: - Summary: Add support for Scala tuples and case classes to Cassandra sink Key: FLINK-4497 URL: https://issues.apache.org/jira/browse/FLINK-4497 Project: Flink

[jira] [Created] (FLINK-4498) Better Cassandra sink documentation

2016-08-25 Thread Elias Levy (JIRA)
Elias Levy created FLINK-4498: - Summary: Better Cassandra sink documentation Key: FLINK-4498 URL: https://issues.apache.org/jira/browse/FLINK-4498 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-4501) Cassandra sink can lose messages

2016-08-25 Thread Elias Levy (JIRA)
Elias Levy created FLINK-4501: - Summary: Cassandra sink can lose messages Key: FLINK-4501 URL: https://issues.apache.org/jira/browse/FLINK-4501 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-4502) Cassandra connector documentation has misleading consistency guarantees

2016-08-25 Thread Elias Levy (JIRA)
Elias Levy created FLINK-4502: - Summary: Cassandra connector documentation has misleading consistency guarantees Key: FLINK-4502 URL: https://issues.apache.org/jira/browse/FLINK-4502 Project: Flink

[jira] [Created] (FLINK-4500) Cassandra sink can lose messages

2016-08-25 Thread Elias Levy (JIRA)
Elias Levy created FLINK-4500: - Summary: Cassandra sink can lose messages Key: FLINK-4500 URL: https://issues.apache.org/jira/browse/FLINK-4500 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-6243) Continuous Joins: True Sliding Window Joins

2017-04-01 Thread Elias Levy (JIRA)
Elias Levy created FLINK-6243: - Summary: Continuous Joins: True Sliding Window Joins Key: FLINK-6243 URL: https://issues.apache.org/jira/browse/FLINK-6243 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-6239) Sharing of State Across Operators

2017-04-01 Thread Elias Levy (JIRA)
Elias Levy created FLINK-6239: - Summary: Sharing of State Across Operators Key: FLINK-6239 URL: https://issues.apache.org/jira/browse/FLINK-6239 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-7364) Log exceptions from user code in streaming jobs

2017-08-03 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7364: - Summary: Log exceptions from user code in streaming jobs Key: FLINK-7364 URL: https://issues.apache.org/jira/browse/FLINK-7364 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-7178) Datadog Metric Reporter Jar is Lacking Dependencies

2017-07-13 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7178: - Summary: Datadog Metric Reporter Jar is Lacking Dependencies Key: FLINK-7178 URL: https://issues.apache.org/jira/browse/FLINK-7178 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-7286) Flink Dashboard fails to display bytes/records received by sources

2017-07-27 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7286: - Summary: Flink Dashboard fails to display bytes/records received by sources Key: FLINK-7286 URL: https://issues.apache.org/jira/browse/FLINK-7286 Project: Flink

[jira] [Created] (FLINK-6419) Better support for CEP quantified conditions in PatternSelect.select

2017-04-28 Thread Elias Levy (JIRA)
Elias Levy created FLINK-6419: - Summary: Better support for CEP quantified conditions in PatternSelect.select Key: FLINK-6419 URL: https://issues.apache.org/jira/browse/FLINK-6419 Project: Flink

[jira] [Created] (FLINK-6420) Cleaner CEP API to specify conditions between events

2017-04-28 Thread Elias Levy (JIRA)
Elias Levy created FLINK-6420: - Summary: Cleaner CEP API to specify conditions between events Key: FLINK-6420 URL: https://issues.apache.org/jira/browse/FLINK-6420 Project: Flink Issue Type

[jira] [Created] (FLINK-6472) BoundedOutOfOrdernessTimestampExtractor does not bound out of orderliness

2017-05-06 Thread Elias Levy (JIRA)
Elias Levy created FLINK-6472: - Summary: BoundedOutOfOrdernessTimestampExtractor does not bound out of orderliness Key: FLINK-6472 URL: https://issues.apache.org/jira/browse/FLINK-6472 Project: Flink

[jira] [Created] (FLINK-7634) Add option to create a savepoint while canceling a job in the dashboard

2017-09-16 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7634: - Summary: Add option to create a savepoint while canceling a job in the dashboard Key: FLINK-7634 URL: https://issues.apache.org/jira/browse/FLINK-7634 Project: Flink

[jira] [Created] (FLINK-7641) Loss of JobManager in HA mode should not cause jobs to fail

2017-09-18 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7641: - Summary: Loss of JobManager in HA mode should not cause jobs to fail Key: FLINK-7641 URL: https://issues.apache.org/jira/browse/FLINK-7641 Project: Flink Issue

[jira] [Created] (FLINK-7640) Dashboard should display information about JobManager cluster in HA mode

2017-09-18 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7640: - Summary: Dashboard should display information about JobManager cluster in HA mode Key: FLINK-7640 URL: https://issues.apache.org/jira/browse/FLINK-7640 Project: Flink

[jira] [Created] (FLINK-7646) Restart failed jobs with configurable parallelism range

2017-09-19 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7646: - Summary: Restart failed jobs with configurable parallelism range Key: FLINK-7646 URL: https://issues.apache.org/jira/browse/FLINK-7646 Project: Flink Issue Type

[jira] [Created] (FLINK-7687) Clarify the master and slaves files are not necessary unless using the cluster start/stop scripts

2017-09-25 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7687: - Summary: Clarify the master and slaves files are not necessary unless using the cluster start/stop scripts Key: FLINK-7687 URL: https://issues.apache.org/jira/browse/FLINK-7687

[jira] [Created] (FLINK-7722) MiniCluster does not appear to honor Log4j settings

2017-09-27 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7722: - Summary: MiniCluster does not appear to honor Log4j settings Key: FLINK-7722 URL: https://issues.apache.org/jira/browse/FLINK-7722 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-7547) o.a.f.s.api.scala.async.AsyncFunction is not declared Serializable

2017-08-28 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7547: - Summary: o.a.f.s.api.scala.async.AsyncFunction is not declared Serializable Key: FLINK-7547 URL: https://issues.apache.org/jira/browse/FLINK-7547 Project: Flink

[jira] [Created] (FLINK-9272) DataDog API "counter" metric type is deprecated

2018-04-28 Thread Elias Levy (JIRA)
Elias Levy created FLINK-9272: - Summary: DataDog API "counter" metric type is deprecated Key: FLINK-9272 URL: https://issues.apache.org/jira/browse/FLINK-9272 Project: Flink

[jira] [Created] (FLINK-9403) Documentation continues to refer to removed methods

2018-05-20 Thread Elias Levy (JIRA)
Elias Levy created FLINK-9403: - Summary: Documentation continues to refer to removed methods Key: FLINK-9403 URL: https://issues.apache.org/jira/browse/FLINK-9403 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-9440) Allow cancelation and reset of timers

2018-05-25 Thread Elias Levy (JIRA)
Elias Levy created FLINK-9440: - Summary: Allow cancelation and reset of timers Key: FLINK-9440 URL: https://issues.apache.org/jira/browse/FLINK-9440 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-9600) Add DataStream transformation variants that pass timestamp to the user function

2018-06-15 Thread Elias Levy (JIRA)
Elias Levy created FLINK-9600: - Summary: Add DataStream transformation variants that pass timestamp to the user function Key: FLINK-9600 URL: https://issues.apache.org/jira/browse/FLINK-9600 Project

[jira] [Created] (FLINK-9450) Job hangs if S3 access it denied during checkpoints

2018-05-26 Thread Elias Levy (JIRA)
Elias Levy created FLINK-9450: - Summary: Job hangs if S3 access it denied during checkpoints Key: FLINK-9450 URL: https://issues.apache.org/jira/browse/FLINK-9450 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-9495) Implement ResourceManager for Kubernetes

2018-06-01 Thread Elias Levy (JIRA)
Elias Levy created FLINK-9495: - Summary: Implement ResourceManager for Kubernetes Key: FLINK-9495 URL: https://issues.apache.org/jira/browse/FLINK-9495 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-9682) Add setDescription to execution environment and display it in the UI

2018-06-27 Thread Elias Levy (JIRA)
Elias Levy created FLINK-9682: - Summary: Add setDescription to execution environment and display it in the UI Key: FLINK-9682 URL: https://issues.apache.org/jira/browse/FLINK-9682 Project: Flink

[jira] [Created] (FLINK-7935) Metrics with user supplied scope variables

2017-10-26 Thread Elias Levy (JIRA)
Elias Levy created FLINK-7935: - Summary: Metrics with user supplied scope variables Key: FLINK-7935 URL: https://issues.apache.org/jira/browse/FLINK-7935 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-8751) Canceling a job results in a InterruptedException in the JM

2018-02-22 Thread Elias Levy (JIRA)
Elias Levy created FLINK-8751: - Summary: Canceling a job results in a InterruptedException in the JM Key: FLINK-8751 URL: https://issues.apache.org/jira/browse/FLINK-8751 Project: Flink Issue

[jira] [Created] (FLINK-8752) ClassNotFoundException when using the user code class loader

2018-02-22 Thread Elias Levy (JIRA)
Elias Levy created FLINK-8752: - Summary: ClassNotFoundException when using the user code class loader Key: FLINK-8752 URL: https://issues.apache.org/jira/browse/FLINK-8752 Project: Flink Issue

[jira] [Created] (FLINK-8352) Flink UI Reports No Error on Job Submission Failures

2018-01-02 Thread Elias Levy (JIRA)
Elias Levy created FLINK-8352: - Summary: Flink UI Reports No Error on Job Submission Failures Key: FLINK-8352 URL: https://issues.apache.org/jira/browse/FLINK-8352 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-8311) Flink needs documentation for network access control

2017-12-22 Thread Elias Levy (JIRA)
Elias Levy created FLINK-8311: - Summary: Flink needs documentation for network access control Key: FLINK-8311 URL: https://issues.apache.org/jira/browse/FLINK-8311 Project: Flink Issue Type

[jira] [Created] (FLINK-8358) Hostname used by DataDog metric reporter is not configurable

2018-01-03 Thread Elias Levy (JIRA)
Elias Levy created FLINK-8358: - Summary: Hostname used by DataDog metric reporter is not configurable Key: FLINK-8358 URL: https://issues.apache.org/jira/browse/FLINK-8358 Project: Flink Issue

[jira] [Created] (FLINK-10037) Document details event time behavior in a single location

2018-08-02 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10037: -- Summary: Document details event time behavior in a single location Key: FLINK-10037 URL: https://issues.apache.org/jira/browse/FLINK-10037 Project: Flink Issue

[jira] [Created] (FLINK-10011) Old job resurrected during HA failover

2018-07-31 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10011: -- Summary: Old job resurrected during HA failover Key: FLINK-10011 URL: https://issues.apache.org/jira/browse/FLINK-10011 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-10098) Programmatically select timer storage backend

2018-08-07 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10098: -- Summary: Programmatically select timer storage backend Key: FLINK-10098 URL: https://issues.apache.org/jira/browse/FLINK-10098 Project: Flink Issue Type

[jira] [Created] (FLINK-10117) REST API for Queryable State

2018-08-09 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10117: -- Summary: REST API for Queryable State Key: FLINK-10117 URL: https://issues.apache.org/jira/browse/FLINK-10117 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-10118) Queryable state MapState entry query

2018-08-09 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10118: -- Summary: Queryable state MapState entry query Key: FLINK-10118 URL: https://issues.apache.org/jira/browse/FLINK-10118 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-9731) Kafka source subtask begins to consume from earliest offset

2018-07-03 Thread Elias Levy (JIRA)
Elias Levy created FLINK-9731: - Summary: Kafka source subtask begins to consume from earliest offset Key: FLINK-9731 URL: https://issues.apache.org/jira/browse/FLINK-9731 Project: Flink Issue

[jira] [Created] (FLINK-8886) Job isolation via scheduling in shared cluster

2018-03-06 Thread Elias Levy (JIRA)
Elias Levy created FLINK-8886: - Summary: Job isolation via scheduling in shared cluster Key: FLINK-8886 URL: https://issues.apache.org/jira/browse/FLINK-8886 Project: Flink Issue Type

[jira] [Created] (FLINK-8844) Export job jar file name or job version property via REST API

2018-03-02 Thread Elias Levy (JIRA)
Elias Levy created FLINK-8844: - Summary: Export job jar file name or job version property via REST API Key: FLINK-8844 URL: https://issues.apache.org/jira/browse/FLINK-8844 Project: Flink Issue

[jira] [Created] (FLINK-10617) Restoring job fails because of slot allocation timeout

2018-10-19 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10617: -- Summary: Restoring job fails because of slot allocation timeout Key: FLINK-10617 URL: https://issues.apache.org/jira/browse/FLINK-10617 Project: Flink Issue

[jira] [Created] (FLINK-10493) Macro generated CaseClassSerializer considered harmful

2018-10-04 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10493: -- Summary: Macro generated CaseClassSerializer considered harmful Key: FLINK-10493 URL: https://issues.apache.org/jira/browse/FLINK-10493 Project: Flink Issue

[jira] [Created] (FLINK-10520) Job save points REST API fails unless parameters are specified

2018-10-09 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10520: -- Summary: Job save points REST API fails unless parameters are specified Key: FLINK-10520 URL: https://issues.apache.org/jira/browse/FLINK-10520 Project: Flink

[jira] [Created] (FLINK-10483) Can't restore from a savepoint even with Allow Non Restored State enabled

2018-10-02 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10483: -- Summary: Can't restore from a savepoint even with Allow Non Restored State enabled Key: FLINK-10483 URL: https://issues.apache.org/jira/browse/FLINK-10483 Project: Flink

[jira] [Created] (FLINK-10460) DataDog reporter JsonMappingException

2018-09-28 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10460: -- Summary: DataDog reporter JsonMappingException Key: FLINK-10460 URL: https://issues.apache.org/jira/browse/FLINK-10460 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-10372) There is no API to configure the timer state backend

2018-09-19 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10372: -- Summary: There is no API to configure the timer state backend Key: FLINK-10372 URL: https://issues.apache.org/jira/browse/FLINK-10372 Project: Flink Issue Type

[jira] [Created] (FLINK-10390) DataDog metric reporter leak warning

2018-09-21 Thread Elias Levy (JIRA)
Elias Levy created FLINK-10390: -- Summary: DataDog metric reporter leak warning Key: FLINK-10390 URL: https://issues.apache.org/jira/browse/FLINK-10390 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-12024) Bump universal Kafka connector to Kafka dependency to 2.2.0

2019-03-26 Thread Elias Levy (JIRA)
Elias Levy created FLINK-12024: -- Summary: Bump universal Kafka connector to Kafka dependency to 2.2.0 Key: FLINK-12024 URL: https://issues.apache.org/jira/browse/FLINK-12024 Project: Flink

[jira] [Created] (FLINK-11794) Allow compression of row format files created by StreamingFileSink

2019-03-01 Thread Elias Levy (JIRA)
Elias Levy created FLINK-11794: -- Summary: Allow compression of row format files created by StreamingFileSink Key: FLINK-11794 URL: https://issues.apache.org/jira/browse/FLINK-11794 Project: Flink

[jira] [Created] (FLINK-11520) Triggers should be provided the window state

2019-02-01 Thread Elias Levy (JIRA)
Elias Levy created FLINK-11520: -- Summary: Triggers should be provided the window state Key: FLINK-11520 URL: https://issues.apache.org/jira/browse/FLINK-11520 Project: Flink Issue Type

[jira] [Created] (FLINK-11517) Inefficient window state access when using RocksDB state backend

2019-02-01 Thread Elias Levy (JIRA)
Elias Levy created FLINK-11517: -- Summary: Inefficient window state access when using RocksDB state backend Key: FLINK-11517 URL: https://issues.apache.org/jira/browse/FLINK-11517 Project: Flink