[jira] [Created] (FLINK-16013) List config option could not be parsed correctly

2020-02-11 Thread Yang Wang (Jira)
Yang Wang created FLINK-16013: - Summary: List config option could not be parsed correctly Key: FLINK-16013 URL: https://issues.apache.org/jira/browse/FLINK-16013 Project: Flink Issue Type: Bug

Re: [VOTE] Release Flink Python API(PyFlink) 1.9.2 to PyPI, release candidate #1

2020-02-11 Thread Becket Qin
+1 (binding) - verified signature - Ran word count example successfully. Thanks, Jiangjie (Becket) Qin On Wed, Feb 12, 2020 at 1:29 PM Jark Wu wrote: > +1 > > - checked/verified signatures and hashes > - Pip installed the package successfully: pip install > apache-flink-1.9.2.tar.gz > - Run

[jira] [Created] (FLINK-16012) Reduce the default number of exclusive buffers from 2 to 1 on receiver side

2020-02-11 Thread zhijiang (Jira)
zhijiang created FLINK-16012: Summary: Reduce the default number of exclusive buffers from 2 to 1 on receiver side Key: FLINK-16012 URL: https://issues.apache.org/jira/browse/FLINK-16012 Project: Flink

[VOTE] Support scalar vectorized Python UDF in PyFlink

2020-02-11 Thread Dian Fu
Hi all, I'd like to start the vote of FLIP-97[1] which is discussed and reached consensus in the discussion thread[2]. The vote will be open for at least 72 hours. Unless there is an objection, I will try to close it by Feb 17, 2020 08:00 UTC if we have received sufficient votes. Regards,

Re: [VOTE] Release Flink Python API(PyFlink) 1.9.2 to PyPI, release candidate #1

2020-02-11 Thread Jark Wu
+1 - checked/verified signatures and hashes - Pip installed the package successfully: pip install apache-flink-1.9.2.tar.gz - Run word count example successfully through the documentation [1]. Best, Jark [1]:

Re: [VOTE] FLIP-55: Introduction of a Table API Java Expression DSL

2020-02-11 Thread Dian Fu
Hi Dawid, Thanks for your reply. I'm also in favor of "col" as a column expression in the Python Table API. Regarding to use "$" in the Java/Scala Table API, I'm fine with it. So +1 from my side. Thanks, Dian > 在 2020年2月11日,下午9:48,Aljoscha Krettek 写道: > > +1 > > Best, > Aljoscha > > On

[jira] [Created] (FLINK-16011) Normalize the within usage in Pattern

2020-02-11 Thread shuai.xu (Jira)
shuai.xu created FLINK-16011: Summary: Normalize the within usage in Pattern Key: FLINK-16011 URL: https://issues.apache.org/jira/browse/FLINK-16011 Project: Flink Issue Type: Improvement

Re: [DISCUSS] FLIP-84: Improve & Refactor execute/sqlQuery/sqlUpdate APIS of TableEnvironment

2020-02-11 Thread godfreyhe
Hi everyone, I'd like to resume the discussion for FlIP-84 [0]. I had updated the document, the mainly changes are: 1. about "`void sqlUpdate(String sql)`" section a) change "Optional executeSql(String sql) throws Exception" to "ResultTable executeStatement(String statement, String jobName)

[jira] [Created] (FLINK-16010) Support notFollowedBy with interval as the last part of a Pattern

2020-02-11 Thread shuai.xu (Jira)
shuai.xu created FLINK-16010: Summary: Support notFollowedBy with interval as the last part of a Pattern Key: FLINK-16010 URL: https://issues.apache.org/jira/browse/FLINK-16010 Project: Flink

[jira] [Created] (FLINK-16009) Native support for the Variable-length and Zig-zag Variable-length integers

2020-02-11 Thread Yun Gao (Jira)
Yun Gao created FLINK-16009: --- Summary: Native support for the Variable-length and Zig-zag Variable-length integers Key: FLINK-16009 URL: https://issues.apache.org/jira/browse/FLINK-16009 Project: Flink

[jira] [Created] (FLINK-16008) Add rules to transpose the join condition of Python Correlate node

2020-02-11 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-16008: Summary: Add rules to transpose the join condition of Python Correlate node Key: FLINK-16008 URL: https://issues.apache.org/jira/browse/FLINK-16008 Project: Flink

[jira] [Created] (FLINK-16007) Add rules to push down the Java Calls contained in Python Correlate node

2020-02-11 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-16007: Summary: Add rules to push down the Java Calls contained in Python Correlate node Key: FLINK-16007 URL: https://issues.apache.org/jira/browse/FLINK-16007 Project:

[jira] [Created] (FLINK-16006) Add host blacklist support for Flink YarnResourceManager

2020-02-11 Thread Zhenqiu Huang (Jira)
Zhenqiu Huang created FLINK-16006: - Summary: Add host blacklist support for Flink YarnResourceManager Key: FLINK-16006 URL: https://issues.apache.org/jira/browse/FLINK-16006 Project: Flink

[jira] [Created] (FLINK-16005) Propagate yarn.application.classpath from client to TaskManager Classpath

2020-02-11 Thread Zhenqiu Huang (Jira)
Zhenqiu Huang created FLINK-16005: - Summary: Propagate yarn.application.classpath from client to TaskManager Classpath Key: FLINK-16005 URL: https://issues.apache.org/jira/browse/FLINK-16005 Project:

[jira] [Created] (FLINK-16004) Exclude flink-rocksdb-state-memory-control-test from the dist

2020-02-11 Thread Yu Li (Jira)
Yu Li created FLINK-16004: - Summary: Exclude flink-rocksdb-state-memory-control-test from the dist Key: FLINK-16004 URL: https://issues.apache.org/jira/browse/FLINK-16004 Project: Flink Issue Type:

[CANCELED][VOTE] Release flink-shaded 10.0, release candidate #1

2020-02-11 Thread Chesnay Schepler
The vote is hereby cancelled since the NOTICE file entries for the new zookeeper modules is not correct. On 11/02/2020 16:43, Hequn Cheng wrote: Hi Chesnay, One thing needs to double-check with you. It seems the zookeeper version is not correct in the NOTICE file for

Re: [VOTE] Release flink-shaded 10.0, release candidate #1

2020-02-11 Thread Hequn Cheng
Hi Chesnay, One thing needs to double-check with you. It seems the zookeeper version is not correct in the NOTICE file for flink-shaded-zookeeper-3.5. The version should be 3.5.6? Best, Hequn On Sun, Feb 9, 2020 at 6:31 PM Chesnay Schepler wrote: > Hi everyone, > Please review and vote on the

[jira] [Created] (FLINK-16003) ShardConsumer errors cannot be logged

2020-02-11 Thread Ori Popowski (Jira)
Ori Popowski created FLINK-16003: Summary: ShardConsumer errors cannot be logged Key: FLINK-16003 URL: https://issues.apache.org/jira/browse/FLINK-16003 Project: Flink Issue Type: Bug

Re: [Discussion] Job generation / submission hooks & Atlas integration

2020-02-11 Thread Gyula Fóra
Hi All! I have made a prototype that simply adds a getPipeline() method to the JobClient interface. Then I could easily implement the Atlas hook using the JobListener interface. I simply check if Pipeline is instanceof StreamGraph and do the logic there. I think this is so far the cleanest

Add support for IAM Roles for Service Accounts in AWS EKS (Web Identity)

2020-02-11 Thread Rafi Aroch
Hi, IAM Roles for Service Accounts have many advantages when deploying Flink on AWS EKS. >From AWS documentation: *With IAM roles for service accounts on Amazon EKS clusters, you can > associate an IAM role with a Kubernetes service account. This service > account can then provide AWS

[jira] [Created] (FLINK-16001) Avoid using Java Streams in construction of ExecutionGraph

2020-02-11 Thread Jiayi Liao (Jira)
Jiayi Liao created FLINK-16001: -- Summary: Avoid using Java Streams in construction of ExecutionGraph Key: FLINK-16001 URL: https://issues.apache.org/jira/browse/FLINK-16001 Project: Flink Issue

Re: [VOTE] Release Flink Python API(PyFlink) 1.9.2 to PyPI, release candidate #1

2020-02-11 Thread Hequn Cheng
+1 (non-binding) - Check signature and checksum. - Install package successfully with Pip under Python 3.7.4. - Run wordcount example successfully under Python 3.7.4. Best, Hequn On Tue, Feb 11, 2020 at 12:17 PM Dian Fu wrote: > +1 (non-binding) > > - Verified the signature and checksum > -

[jira] [Created] (FLINK-16000) Move "Project Build Setup" to "Getting Started" in documentation

2020-02-11 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-16000: Summary: Move "Project Build Setup" to "Getting Started" in documentation Key: FLINK-16000 URL: https://issues.apache.org/jira/browse/FLINK-16000 Project:

Re: [VOTE] FLIP-55: Introduction of a Table API Java Expression DSL

2020-02-11 Thread Aljoscha Krettek
+1 Best, Aljoscha On 11.02.20 11:17, Jingsong Li wrote: Thanks Dawid for your explanation, +1 for vote. So I am big +1 to accepting java.lang.Object in the Java DSL, without scala implicit conversion, a lot of "lit" look unfriendly to users. Best, Jingsong Lee On Tue, Feb 11, 2020 at 6:07

[jira] [Created] (FLINK-15999) Extract “Concepts” material from API/Library sections and start proper concepts section

2020-02-11 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-15999: Summary: Extract “Concepts” material from API/Library sections and start proper concepts section Key: FLINK-15999 URL: https://issues.apache.org/jira/browse/FLINK-15999

[jira] [Created] (FLINK-15998) Revert rename of "Job Cluster" to "Application Cluster" in documentation

2020-02-11 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-15998: Summary: Revert rename of "Job Cluster" to "Application Cluster" in documentation Key: FLINK-15998 URL: https://issues.apache.org/jira/browse/FLINK-15998

[jira] [Created] (FLINK-15997) Make documentation 404 page look like a documentation page

2020-02-11 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-15997: Summary: Make documentation 404 page look like a documentation page Key: FLINK-15997 URL: https://issues.apache.org/jira/browse/FLINK-15997 Project: Flink

[jira] [Created] (FLINK-15996) Provide DataStream API for JDBC sinks

2020-02-11 Thread Roman Khachatryan (Jira)
Roman Khachatryan created FLINK-15996: - Summary: Provide DataStream API for JDBC sinks Key: FLINK-15996 URL: https://issues.apache.org/jira/browse/FLINK-15996 Project: Flink Issue Type:

[jira] [Created] (FLINK-15995) TO_BASE64 sql operator change operandTypeChecker from OperandTypes.ANY to OperandTypes.family(SqlTypeFamily.STRING)

2020-02-11 Thread hailong wang (Jira)
hailong wang created FLINK-15995: Summary: TO_BASE64 sql operator change operandTypeChecker from OperandTypes.ANY to OperandTypes.family(SqlTypeFamily.STRING) Key: FLINK-15995 URL:

[jira] [Created] (FLINK-15994) Support byte array arguments for FROM_BASE64 function

2020-02-11 Thread hailong wang (Jira)
hailong wang created FLINK-15994: Summary: Support byte array arguments for FROM_BASE64 function Key: FLINK-15994 URL: https://issues.apache.org/jira/browse/FLINK-15994 Project: Flink Issue

[jira] [Created] (FLINK-15993) Add timeout to 404 documentation redirect, add explanation

2020-02-11 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-15993: Summary: Add timeout to 404 documentation redirect, add explanation Key: FLINK-15993 URL: https://issues.apache.org/jira/browse/FLINK-15993 Project: Flink

Re: [DISCUSS] Support scalar vectorized Python UDF in PyFlink

2020-02-11 Thread Dian Fu
Hi all, Thanks you all participating this discussion and sharing your thoughts. It seems that we have reached consensus on the design now. I will start a VOTE thread if there are no other feedbacks. Thanks, Dian On Tue, Feb 11, 2020 at 10:23 AM Dian Fu wrote: > Hi Jingsong, > > You're right.

Re: [VOTE] FLIP-55: Introduction of a Table API Java Expression DSL

2020-02-11 Thread Jingsong Li
Thanks Dawid for your explanation, +1 for vote. So I am big +1 to accepting java.lang.Object in the Java DSL, without scala implicit conversion, a lot of "lit" look unfriendly to users. Best, Jingsong Lee On Tue, Feb 11, 2020 at 6:07 PM Dawid Wysakowicz wrote: > Hi, > > To answer some of the

[jira] [Created] (FLINK-15992) Incorrect classloader when finding TableFactory

2020-02-11 Thread Victor Wong (Jira)
Victor Wong created FLINK-15992: --- Summary: Incorrect classloader when finding TableFactory Key: FLINK-15992 URL: https://issues.apache.org/jira/browse/FLINK-15992 Project: Flink Issue Type:

[jira] [Created] (FLINK-15991) Create Chinese documentation for FLIP-49 TM memory model

2020-02-11 Thread Andrey Zagrebin (Jira)
Andrey Zagrebin created FLINK-15991: --- Summary: Create Chinese documentation for FLIP-49 TM memory model Key: FLINK-15991 URL: https://issues.apache.org/jira/browse/FLINK-15991 Project: Flink

Re: [VOTE] FLIP-55: Introduction of a Table API Java Expression DSL

2020-02-11 Thread Dawid Wysakowicz
Hi, To answer some of the questions: @Jingsong We use Objects in the java API to make it possible to use raw Objects without the need to wrap them in literals. If an expression is passed it is used as is. If anything else is used, it is assumed to be an literal and is wrapped into a literal.

Re: [DISCUSS] have separate Flink distributions with built-in Hive dependencies

2020-02-11 Thread Stephan Ewen
IIRC, Guowei wants to work on supporting Table API connectors in Plugins. With that, we could have the Hive dependency as a plugin, avoiding dependency conflicts. On Thu, Feb 6, 2020 at 1:11 PM Jingsong Li wrote: > Hi Stephan, > > Good idea. Just like hadoop, we can have flink-shaded-hive-uber.

Re: [DISCUSS] Drop connectors for Elasticsearch 2.x and 5.x

2020-02-11 Thread Chesnay Schepler
I suppose the downside in an HTTP ES sink is that you don't get _any_ form of high-level API from ES, and we'd have to manually build an HTTP request that matches the ES format. Of course you also lose any client-side verification that the clients did, if there is any (but I guess the API

[jira] [Created] (FLINK-15990) Remove register source and sink in ConnectTableDescriptor

2020-02-11 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-15990: Summary: Remove register source and sink in ConnectTableDescriptor Key: FLINK-15990 URL: https://issues.apache.org/jira/browse/FLINK-15990 Project: Flink

Re: [VOTE] FLIP-55: Introduction of a Table API Java Expression DSL

2020-02-11 Thread Dian Fu
Hi Dawid, Thanks for driving this feature. The design looks very well for me overall. I have only one concern: $ is not allowed to be used in the identifier of Python and so we have to come out with another symbol when aligning this feature in the Python Table API. I noticed that there are

[DISCUSS] Support notFollowedBy with interval as the last part of a Pattern

2020-02-11 Thread Shuai Xu
Hi all, CEP is broadly used in more and more applications now. In may cases, users need to use the pattern CEP.begin().notFollowedBy(). For example, they may want to get the uses who created an oder but didn't pay in 10 minutes and so on. However, CEP doesn't support notFollowBy() as the last

Re: [VOTE] Release 1.10.0, release candidate #3

2020-02-11 Thread Andrey Zagrebin
Hi, @Jingsong Lee Regarding "OutOfMemoryError: Direct buffer memory" in FileChannelBoundedData$FileBufferReader I saw you created a specific issue issue: https://issues.apache.org/jira/browse/FLINK-15981 In general, I think we could rewrap this error in

[jira] [Created] (FLINK-15989) Rewrap OutOfMemoryError in allocateUnpooledOffHeap with better message

2020-02-11 Thread Andrey Zagrebin (Jira)
Andrey Zagrebin created FLINK-15989: --- Summary: Rewrap OutOfMemoryError in allocateUnpooledOffHeap with better message Key: FLINK-15989 URL: https://issues.apache.org/jira/browse/FLINK-15989

Re: [DISCUSS] Drop connectors for Elasticsearch 2.x and 5.x

2020-02-11 Thread Stephan Ewen
+1 to drop ES 2.x - unsure about 5.x (makes sense to get more user input for that one). @Itamar - if you would be interested in contributing a "universal" or "cross version" ES connector, that could be very interesting. Do you know if there are known performance issues or feature restrictions

[jira] [Created] (FLINK-15988) Make JsonRowSerializationSchema's constructor private

2020-02-11 Thread Benchao Li (Jira)
Benchao Li created FLINK-15988: -- Summary: Make JsonRowSerializationSchema's constructor private Key: FLINK-15988 URL: https://issues.apache.org/jira/browse/FLINK-15988 Project: Flink Issue