[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-07-08 Thread suez1224
GitHub user suez1224 reopened a pull request: https://github.com/apache/flink/pull/6201 [FLINK-8866][Table API & SQL] Add support for unified table sink instantiation **(The sections below can be removed for hotfixes of typos)** ## What is the purpose of the ch

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-07-08 Thread suez1224
Github user suez1224 closed the pull request at: https://github.com/apache/flink/pull/6201 ---

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-07-07 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r200806113 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala --- @@ -39,9 +39,12 @@ abstract class

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-07-07 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r200806045 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala --- @@ -27,6 +28,8 @@ class FileSystem extends

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-07-07 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r200805902 --- Diff: flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java --- @@ -56,23 +58,44 @@ public Environment

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-07-07 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r200805881 --- Diff: flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java --- @@ -56,23 +58,44 @@ public Environment

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-07-03 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r199951101 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connector/TableConnectorFactory.scala --- @@ -16,21 +16,18

[GitHub] flink issue #6201: [FLINK-8866][Table API & SQL] Add support for unified tab...

2018-07-03 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/6201 @twalthr @fhueske sounds good to me. We can do that in a follow-up issue for `from-source`, and we will not support `from-source` in this PR. ---

[GitHub] flink issue #6201: [FLINK-8866][Table API & SQL] Add support for unified tab...

2018-07-03 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/6201 @fhueske @twalthr thanks for the comments. In `from-source`, the only system i know of is Kafka10 or Kafka11, which support writing record along with timestamp. To support `from-source` in table

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-06-29 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r199066851 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala --- @@ -0,0 +1,151

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-06-29 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r199066578 --- Diff: flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java --- @@ -321,6 +327,18 @@ public

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-06-29 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r199065915 --- Diff: flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java --- @@ -321,6 +327,18 @@ public

[GitHub] flink issue #6201: [FLINK-8866][Table API & SQL] Add support for unified tab...

2018-06-29 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/6201 @twalthr , for sink only table, I dont think the user need to define any rowtimes on it, since it will never use as a source. For table as both source and sink, when registering it as sink, I think

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-06-27 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r198678147 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connector/TableSinkFactoryServiceTest.scala --- @@ -0,0 +1,84

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-06-27 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r198677922 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connector/TableConnectorFactory.scala --- @@ -16,21 +16,18

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-06-27 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r198677762 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connector/TableConnectorFactory.scala --- @@ -16,21 +16,18

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-06-27 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6201#discussion_r198677754 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connector/TableConnectorFactory.scala --- @@ -16,21 +16,18

[GitHub] flink pull request #6201: [FLINK-8866][Table API & SQL] Add support for unif...

2018-06-22 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/6201 [FLINK-8866][Table API & SQL] Add support for unified table sink instantiation **(The sections below can be removed for hotfixes of typos)** ## What is the purpose of the ch

[GitHub] flink pull request #6121: [FLINK-9482] [table] EXTRACT function argument val...

2018-06-07 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6121#discussion_r193637621 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/validation/ScalarFunctionsValidationTest.scala --- @@ -98,16

[GitHub] flink pull request #6082: [FLINK-9444][table] KafkaAvroTableSource failed to...

2018-06-05 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6082#discussion_r192955182 --- Diff: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroRecordClassConverter.java --- @@ -73,9 +75,37 @@ private

[GitHub] flink issue #6082: [FLINK-9444][table] KafkaAvroTableSource failed to work f...

2018-05-31 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/6082 Thanks for the PR, @tragicjun. I will take a look in the next few days. ---

[GitHub] flink pull request #6100: [FLINK-9430][Table API & SQL] add cast rule in SQL...

2018-05-30 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/6100 [FLINK-9430][Table API & SQL] add cast rule in SQL to cast Object to other assignable types ## What is the purpose of the change Add cast rule in SQL to cast Object to o

[GitHub] flink pull request #5306: [FLINK-8390][security]remove unused integration te...

2018-05-23 Thread suez1224
Github user suez1224 closed the pull request at: https://github.com/apache/flink/pull/5306 ---

[GitHub] flink issue #5306: [FLINK-8390][security]remove unused integration test code

2018-05-23 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5306 @zentol , you are right, https://issues.apache.org/jira/browse/FLINK-8286 already did the removal. ---

[GitHub] flink pull request #5619: [FLINK-8838] [table] Add Support for UNNEST a Mult...

2018-05-17 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5619#discussion_r189167881 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/ExplodeFunctionUtil.scala --- @@ -18,74 +18,244

[GitHub] flink pull request #5619: [FLINK-8838] [table] Add Support for UNNEST a Mult...

2018-05-17 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5619#discussion_r189144224 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/ExplodeFunctionUtil.scala --- @@ -18,74 +18,244

[GitHub] flink pull request #5619: [FLINK-8838] [table] Add Support for UNNEST a Mult...

2018-05-17 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5619#discussion_r189144701 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/ExplodeFunctionUtil.scala --- @@ -18,74 +18,244

[GitHub] flink pull request #5619: [FLINK-8838] [table] Add Support for UNNEST a Mult...

2018-05-17 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5619#discussion_r189144263 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/ExplodeFunctionUtil.scala --- @@ -18,74 +18,244

[GitHub] flink pull request #5619: [FLINK-8838] [table] Add Support for UNNEST a Mult...

2018-05-17 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5619#discussion_r189145367 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala --- @@ -367,6 +367,36 @@ class

[GitHub] flink pull request #6026: [FLINK-9384]KafkaAvroTableSource failed to work du...

2018-05-16 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6026#discussion_r188786194 --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java --- @@ -81,7

[GitHub] flink pull request #6026: [FLINK-9384]KafkaAvroTableSource failed to work du...

2018-05-16 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/6026#discussion_r188786307 --- Diff: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDeserializationSchema.java --- @@ -110,6 +116,8 @@ private void

[GitHub] flink issue #5896: [FLINK-8286][Security] Fix kerberos security configuratio...

2018-05-09 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5896 @aljoscha that's a very good catch. I added another commit to make sure the call order is always correct (https://github.com/apache/flink/pull/5896/commits/1298b04ea80b9160ac7e4ae8a171f80112a95945

[GitHub] flink pull request #5901: [FLINK-9235][Security] Add integration tests for Y...

2018-05-06 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5901#discussion_r186298337 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java --- @@ -85,54 +88,75 @@ public void

[GitHub] flink pull request #5896: [FLINK-8286][Security] Fix kerberos security confi...

2018-05-02 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5896#discussion_r185606093 --- Diff: flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunnerFactory.java --- @@ -73,61 +105,68 @@ public static void main(String[] args

[GitHub] flink pull request #5896: [FLINK-8286][Security] Fix kerberos security confi...

2018-04-30 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5896#discussion_r185134529 --- Diff: flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunnerFactory.java --- @@ -73,61 +105,68 @@ public static void main(String[] args

[GitHub] flink pull request #5896: [FLINK-8286][Security] Fix kerberos security confi...

2018-04-30 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5896#discussion_r185135031 --- Diff: flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunnerFactory.java --- @@ -73,61 +105,68 @@ public static void main(String[] args

[GitHub] flink pull request #5913: [FLINK-9181] [docs] [sql-client] Add documentation...

2018-04-29 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5913#discussion_r184872369 --- Diff: docs/dev/table/sqlClient.md --- @@ -0,0 +1,539 @@ +--- +title: "SQL Client" +nav-parent_id: tableapi +nav-pos: 100

[GitHub] flink pull request #5940: [FLINK-8690][table]Support group window distinct a...

2018-04-28 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5940#discussion_r184866337 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonAggregate.scala --- @@ -49,11 +49,16 @@ trait CommonAggregate

[GitHub] flink pull request #5940: [FLINK-8690][table]Support group window distinct a...

2018-04-28 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5940#discussion_r184866214 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalAggregate.scala --- @@ -82,7 +82,7 @@ private

[GitHub] flink pull request #5901: [FLINK-9235][Security] Add integration tests for Y...

2018-04-23 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5901 [FLINK-9235][Security] Add integration tests for YARN kerberos integration. ## What is the purpose of the change Add integration tests for YARN kerberos integration. ## Brief

[GitHub] flink pull request #5896: [FLINK-8286][Security] Fix kerberos security confi...

2018-04-23 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5896 [FLINK-8286][Security] Fix kerberos security configuration for YarnTaskExecutor ## What is the purpose of the change Fix broken YARN kerberos integration for flip-6

[GitHub] flink pull request #5895: Fix kerberos security configuration for YarnTaskEx...

2018-04-23 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5895 Fix kerberos security configuration for YarnTaskExecutor ## What is the purpose of the change Fix broken YARN kerberos integration for flip-6. ## Brief change log

[GitHub] flink pull request #5895: Fix kerberos security configuration for YarnTaskEx...

2018-04-23 Thread suez1224
Github user suez1224 closed the pull request at: https://github.com/apache/flink/pull/5895 ---

[GitHub] flink pull request #5867: [FLINK-8686] [sql-client] Improve basic embedded S...

2018-04-19 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5867#discussion_r182650740 --- Diff: flink-libraries/flink-sql-client/bin/sql-client.sh --- @@ -44,20 +44,32 @@ bin=`dirname "$target"` . "

[GitHub] flink issue #5867: [FLINK-8686] [sql-client] Improve basic embedded SQL clie...

2018-04-19 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5867 Hi @twalthr , IMO, it's better to seperate the configuration of SQL client from the configuration of the Flink job. From the user's perspective, many of them probably do not need to know

[GitHub] flink pull request #5758: [FLINK-9059][Table API & SQL] add table type attri...

2018-04-06 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5758#discussion_r179686465 --- Diff: flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java --- @@ -29,38 +30,47

[GitHub] flink pull request #5758: [FLINK-9059][Table API & SQL] add table type attri...

2018-04-06 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5758#discussion_r179686471 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptor.scala --- @@ -0,0 +1,75

[GitHub] flink pull request #5758: [FLINK-9059][Table API & SQL] add table type attri...

2018-04-03 Thread suez1224
GitHub user suez1224 reopened a pull request: https://github.com/apache/flink/pull/5758 [FLINK-9059][Table API & SQL] add table type attribute; replace "sources" with "tables" in environm… ## What is the purpose of the change Add support for unifi

[GitHub] flink pull request #5758: [FLINK-9059][Table API & SQL] add table type attri...

2018-04-03 Thread suez1224
Github user suez1224 closed the pull request at: https://github.com/apache/flink/pull/5758 ---

[GitHub] flink issue #5688: [FLINK-6968][Table API & SQL] Add Queryable table sink.

2018-03-30 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5688 Could you please rebase your pr to resolve conflict? Thanks. ---

[GitHub] flink pull request #5758: [FLINK-9059][Table API & SQL] add table type attri...

2018-03-30 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5758#discussion_r178330131 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptor.scala --- @@ -0,0 +1,75

[GitHub] flink pull request #5758: [FLINK-9059][Table API & SQL] add table type attri...

2018-03-30 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5758#discussion_r178244750 --- Diff: flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java --- @@ -102,10 +112,10 @@ public static

[GitHub] flink pull request #5758: [FLINK-9059][Table API & SQL] add table type attri...

2018-03-30 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5758#discussion_r178244473 --- Diff: flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java --- @@ -29,38 +30,47

[GitHub] flink pull request #5792: [Flink-8563][Table API & SQL] add unittest for con...

2018-03-29 Thread suez1224
Github user suez1224 closed the pull request at: https://github.com/apache/flink/pull/5792 ---

[GitHub] flink pull request #5792: [Flink-8563][Table API & SQL] add unittest for con...

2018-03-29 Thread suez1224
GitHub user suez1224 reopened a pull request: https://github.com/apache/flink/pull/5792 [Flink-8563][Table API & SQL] add unittest for consecutive dot access of composite array element in SQL ## What is the purpose of the change add unittest for consecutive dot ac

[GitHub] flink issue #5792: [Flink-8563][Table API & SQL] add unittest for consecutiv...

2018-03-29 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5792 retry build ---

[GitHub] flink issue #5791: [FLINK-8507][Table API & SQL] upgrade calcite dependency ...

2018-03-29 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5791 Close and reopen to trigger travis. SavepointITCase.testSavepointForJobWithIteration is unstable. ---

[GitHub] flink pull request #5791: [FLINK-8507][Table API & SQL] upgrade calcite depe...

2018-03-29 Thread suez1224
GitHub user suez1224 reopened a pull request: https://github.com/apache/flink/pull/5791 [FLINK-8507][Table API & SQL] upgrade calcite dependency to 1.16 ## What is the purpose of the change Upgrade Flink table's Calcite dependency to 1.16. ## B

[GitHub] flink pull request #5791: [FLINK-8507][Table API & SQL] upgrade calcite depe...

2018-03-29 Thread suez1224
Github user suez1224 closed the pull request at: https://github.com/apache/flink/pull/5791 ---

[GitHub] flink pull request #5794: [Flink-8509][Table API & SQL] Remove SqlGroupedWin...

2018-03-29 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5794 [Flink-8509][Table API & SQL] Remove SqlGroupedWindowFunction copy from flink ## What is the purpose of the change Remove SqlGroupedWindowFunction copy from flink. This dep

[GitHub] flink pull request #5793: [Flink-8508][Table API & SQL] Remove RexSimplify c...

2018-03-29 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5793 [Flink-8508][Table API & SQL] Remove RexSimplify copy from flink ## What is the purpose of the change Remove RexSimplify copy from flink. This depends on https://github.com/ap

[GitHub] flink pull request #5792: [Flink-8563][Table API & SQL] add unittest for con...

2018-03-29 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5792 [Flink-8563][Table API & SQL] add unittest for consecutive dot access of composite array element in SQL ## What is the purpose of the change add unittest for consecutive dot ac

[GitHub] flink pull request #5791: [FLINK-8507][Table API & SQL] upgrade calcite depe...

2018-03-29 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5791 [FLINK-8507][Table API & SQL] upgrade calcite dependency to 1.16 ## What is the purpose of the change Upgrade Flink table's Calcite dependency to 1.16. ## Brief ch

[GitHub] flink pull request #5758: [FLINK-9059][Table API & SQL] add table type attri...

2018-03-23 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5758 [FLINK-9059][Table API & SQL] add table type attribute; replace "sources" with "tables" in environm… ## What is the purpose of the change Add support for unifi

[GitHub] flink issue #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecuredITCase

2018-03-21 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5416 @tillrohrmann addressed your comments. PTAL. ---

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-21 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r176231789 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java --- @@ -465,27 +514,23 @@ private static void start(YarnConfiguration conf

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-21 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r176230467 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java --- @@ -106,8 +106,16 @@ public void testDetachedMode() throws

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-21 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r176209369 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java --- @@ -105,7 +105,7 @@ public void testDetachedMode() throws

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-21 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r176209223 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java --- @@ -515,28 +514,23 @@ private static void start(YarnConfiguration conf

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-21 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r176209172 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java --- @@ -465,27 +514,23 @@ private static void start(YarnConfiguration conf

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-20 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r175677410 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java --- @@ -105,7 +105,7 @@ public void testDetachedMode() throws

[GitHub] flink issue #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecuredITCase

2018-03-20 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5416 @tillrohrmann what signal should I wait for? I can address the flip6 mode test in the end-to--end test JIRA https://issues.apache.org/jira/browse/FLINK-8981. What do you think? ---

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-20 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r175672004 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java --- @@ -206,7 +209,7 @@ public void checkClusterEmpty() throws IOException

[GitHub] flink issue #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecuredITCase

2018-03-19 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5416 Also, I've disabled the test for flip6 mode in order to make the test pass. Can you explain a bit on why this code block (https://github.com/apache/flink/blob

[GitHub] flink issue #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecuredITCase

2018-03-19 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5416 Thanks a lot, @tillrohrmann, for looking into the PR. I've rebased and addressed your comment. Please take another look. ---

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-19 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r175342198 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java --- @@ -379,6 +382,50 @@ public boolean accept(File dir, String name

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-19 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r175341644 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java --- @@ -206,7 +209,7 @@ public void checkClusterEmpty() throws IOException

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-18 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r175325407 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java --- @@ -97,6 +98,15 @@ public static void

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-03-18 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5416#discussion_r175325243 --- Diff: flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java --- @@ -105,7 +105,7 @@ public void testDetachedMode() throws

[GitHub] flink pull request #5706: [FLINK-8903] [table] Fix VAR_SAMP, VAR_POP, STDDEV...

2018-03-16 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5706#discussion_r175183811 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalWindowAggregate.scala --- @@ -103,6 +106,19

[GitHub] flink pull request #5706: [FLINK-8903] [table] Fix VAR_SAMP, VAR_POP, STDEV_...

2018-03-15 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5706#discussion_r174962566 --- Diff: flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java --- @@ -0,0 +1,590

[GitHub] flink pull request #5706: [FLINK-8903] [table] Fix VAR_SAMP, VAR_POP, STDEV_...

2018-03-15 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5706#discussion_r174965171 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalWindowAggregate.scala --- @@ -103,6 +106,19

[GitHub] flink issue #5638: [FLINK-6924][table]ADD LOG(X) supported in TableAPI

2018-03-10 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5638 The PR looks good. Could you please add documentation in docs/dev/table/tableApi.md for both java and scala? Thanks. ---

[GitHub] flink issue #5272: [Flink-8397][Connectors]Support Row type for Cassandra Ou...

2018-02-15 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5272 Thanks a lot, @fhueske, I'll do that next time. ---

[GitHub] flink issue #5272: [Flink-8397][Connectors]Support Row type for Cassandra Ou...

2018-02-14 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5272 @fhueske can you help take another look? Thanks. ---

[GitHub] flink pull request #5272: [Flink-8397][Connectors]Support Row type for Cassa...

2018-02-14 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5272#discussion_r168338536 --- Diff: flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java --- @@ -95,15

[GitHub] flink pull request #5272: [Flink-8397][Connectors]Support Row type for Cassa...

2018-02-14 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5272#discussion_r168330940 --- Diff: flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraTupleOutputFormat.java --- @@ -0,0

[GitHub] flink pull request #5272: [Flink-8397][Connectors]Support Row type for Cassa...

2018-02-14 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5272#discussion_r168331011 --- Diff: flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java --- @@ -37,11

[GitHub] flink pull request #5272: [Flink-8397][Connectors]Support Row type for Cassa...

2018-02-14 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5272#discussion_r168330917 --- Diff: flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraRowOutputFormat.java --- @@ -0,0

[GitHub] flink pull request #5272: [Flink-8397][Connectors]Support Row type for Cassa...

2018-02-14 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5272#discussion_r168330890 --- Diff: flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java

[GitHub] flink pull request #5272: [Flink-8397][Connectors]Support Row type for Cassa...

2018-02-14 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5272#discussion_r168330869 --- Diff: flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java

[GitHub] flink issue #5274: [FLINK-8401][Cassandra Connector]Refactor CassandraOutput...

2018-02-13 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5274 @tillrohrmann , can you help take another look? ---

[GitHub] flink pull request #5472: [FLINK-7658][Table API & SQL] support Collect aggr...

2018-02-13 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5472 [FLINK-7658][Table API & SQL] support Collect aggregate function in Table API ## What is the purpose of the change Add Collect aggregate function for Table API. ## B

[GitHub] flink issue #5172: [FLINK-8275] [Security] fix keytab local path in YarnTask...

2018-02-07 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5172 It's already merged to master, and should be available in Flink 1.4.1. On Wed, Feb 7, 2018 at 11:11 PM, toggm <notificati...@github.com> wrote: > What happended with thi

[GitHub] flink pull request #5416: [FLINK-8562] [Security] Fix YARNSessionFIFOSecured...

2018-02-06 Thread suez1224
GitHub user suez1224 opened a pull request: https://github.com/apache/flink/pull/5416 [FLINK-8562] [Security] Fix YARNSessionFIFOSecuredITCase ## What is the purpose of the change Fix YARNSessionFIFOSecuredITCase. It's currently broken. ## Brief change log

[GitHub] flink issue #5172: [FLINK-8275] [Security] fix keytab local path in YarnTask...

2018-02-05 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5172 @tzulitai thanks a lot for the refactoring, the commit looks good to me. ---

[GitHub] flink issue #5367: [FLINK-7923][Table API & SQL] Support field access of com...

2018-02-02 Thread suez1224
Github user suez1224 commented on the issue: https://github.com/apache/flink/pull/5367 @hequn8128 @twalthr could you please take another look? I've refactored the code to reuse the generateFieldAccess() code, also added null test. Nested tuple wont work now due to https

[GitHub] flink pull request #5367: [FLINK-7923][Table API & SQL] Support field access...

2018-02-01 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5367#discussion_r165560261 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/CalcTest.scala --- @@ -0,0 +1,47 @@ +/* + * Licensed

[GitHub] flink pull request #5367: [FLINK-7923][Table API & SQL] Support field access...

2018-02-01 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5367#discussion_r165560235 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala --- @@ -984,6 +987,63 @@ object

[GitHub] flink pull request #5367: [FLINK-7923][Table API & SQL] Support field access...

2018-02-01 Thread suez1224
Github user suez1224 commented on a diff in the pull request: https://github.com/apache/flink/pull/5367#discussion_r165527528 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala --- @@ -984,6 +987,63 @@ object

  1   2   >