You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/23 12:12:12 UTC

[GitHub] [flink] aalexandrov opened a new pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

aalexandrov opened a new pull request #13770:
URL: https://github.com/apache/flink/pull/13770


   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   Enable the use of the `KinesisProducer` and `KinesisConsumer` as Table API sources and sinks through a new `kinesis` connector type following the [User-defined Sources & Sinks](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/sourceSinks.html) documentation and using the Kafka and Elasticsearch factories as a reference implementation.. 
   
   ## Brief change log
   
   This change adds an implementation, unit-tests, and docs for the following factories:
   
   - `KinesisDynamicSource` - produces `FlinkKinesisConsumer`instances.
   - `KinesisDynamicSink` - produces `FlinkKinesisProducer` instances.
   - `KinesisDynamicTableFactory` - produces instances of the above two factories.
   
   The `KinesisDynamicSink` implements the optional `SupportsPartitioning` ability interface through
   a `RowDataFieldsKinesisPartitioner` class.
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
     - Added `KinesisDynamicTableFactoryTest` unit test for the `KinesisDynamicSource`, `KinesisDynamicSink`, and `KinesisDynamicTableFactory` components. 
     - Added `RowDataFieldsKinesisPartitionerTest` unit test for the `RowDataFieldsKinesisPartitioner` component. 
     - Manually verified the change by running a stateful different program using the enabled SQL syntax against Kinesis Data Streams ingesting data generated from [the `amazon-kinesis-data-generator` Web UI](https://awslabs.github.io/amazon-kinesis-data-generator/web/producer.html). Verified that data written by a Flink sink to target Kinesis streams is partitioend correctly if the corresponding SQL table is defined with the `PARTITIONED BY` clause.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no (changes in the pom.xml are removing obsolete test dependencies and reorganizing the existing ones in more readable blocks).
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: yes (the newly added `RandomKinesisPartitioner` and `FixedKinesisPartitioner` can be used outside of the Table ecosystem in the lower-level DataStream API.
     - The serializers: no.
     - The runtime per-record code paths (performance sensitive): not changing existing paths, but a new path is added in the `RowDataFieldsKinesisPartitioner#getPartitionId` method.
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: no.
     - The S3 file system connector: no.
   
   ## Documentation
   
     - Does this pull request introduce a new feature? yes
     - If yes, how is the feature documented? 
       - Added a new documentation section under `./docs/dev/table/connectors/kinesis.md`) (TODO, should have it later today).
       - Added JavaDocs to all new classes and unit tests.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 115d8d0f08bbdee1ca6becd32a0ca8698b486d72 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810) 
   * 8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517302306



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitionerTest.java
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.temporal.ChronoUnit;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataFieldsKinesisPartitioner.MAX_PARTITION_KEY_LENGTH;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for {@link RowDataFieldsKinesisPartitioner}.
+ */
+public class RowDataFieldsKinesisPartitionerTest extends TestLogger {
+
+	/**
+	 * Table name to use for the tests.
+	 */
+	private static final String TABLE_NAME = "click_stream";
+
+	/**
+	 * Table schema to use for the tests.
+	 */
+	private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+		.field("time", DataTypes.TIMESTAMP(3))
+		.field("date", DataTypes.DATE(), "DATE(`time`)")
+		.field("ip", DataTypes.VARCHAR(16))
+		.field("route", DataTypes.STRING())
+		.build();
+
+	/**
+	 * A {@code PARTITION BY(date, ip)} clause to use for the positive tests.
+	 */
+	private static final List<String> PARTITION_BY_DATE_AND_IP = Arrays.asList("date", "ip");
+
+	/**
+	 * A list of field delimiters to use in the tests.
+	 */
+	private static final List<String> FIELD_DELIMITERS = Arrays.asList("", "|", ",", "--");
+
+	/**
+	 * A {@code PARTITION BY(route)} clause to use for the positive tests.
+	 */
+	private static final List<String> PARTITION_BY_ROUTE = Collections.singletonList("route");
+
+	/**
+	 * Some not-so-random {@link LocalDateTime} instances to use for sample {@link RowData}
+	 * elements in the tests.
+	 */
+	private static final List<LocalDateTime> DATE_TIMES = Arrays.asList(
+		LocalDateTime.of(2014, 10, 22, 14, 0),
+		LocalDateTime.of(2015, 12, 14, 14, 0));
+
+	/**
+	 * A default IP to use for sample {@link RowData} elements in the tests.
+	 */
+	private static final String IP = "255.255.255.255";
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodPartitionerWithDynamicPrefix() {
+		CatalogTable table = createTable(defaultTableOptions(), PARTITION_BY_DATE_AND_IP);
+
+		for (String delimiter : FIELD_DELIMITERS) {
+			RowDataFieldsKinesisPartitioner partitioner =
+				new RowDataFieldsKinesisPartitioner(table, delimiter);
+
+			for (LocalDateTime time : DATE_TIMES) {
+				String expectedKey = String.join(delimiter, String.valueOf(days(time)), IP);
+				String actualKey = partitioner.getPartitionId(createElement(time, IP));
+
+				assertEquals(expectedKey, actualKey);
+			}
+		}
+	}
+
+	@Test
+	public void testGoodPartitionerWithDynamicPrefixExceedingMaxLength() {
+		CatalogTable table = createTable(defaultTableOptions(), PARTITION_BY_ROUTE);
+		RowDataFieldsKinesisPartitioner partitioner = new RowDataFieldsKinesisPartitioner(table);
+
+		String ip = "255.255.255.255";
+		String route = "http://www.very-" + repeat("long-", 50) + "address.com/home";
+		String expectedKey = route.substring(0, MAX_PARTITION_KEY_LENGTH);
+
+		for (LocalDateTime time : DATE_TIMES) {
+			String actualKey = partitioner.getPartitionId(createElement(time, ip, route));
+			assertEquals(expectedKey, actualKey);
+		}
+	}
+
+	@Test
+	public void testGoodPartitionerWithStaticPrefix() {
+		CatalogTable table = createTable(defaultTableOptions(), PARTITION_BY_DATE_AND_IP);
+
+		// fixed prefix
+		int date = days(DATE_TIMES.get(0));
+
+		for (String delimiter : FIELD_DELIMITERS) {
+			RowDataFieldsKinesisPartitioner partitioner =
+				new RowDataFieldsKinesisPartitioner(table, delimiter);
+
+			partitioner.setFixedPrefix(new HashMap<String, String>() {{
+				put("date", String.valueOf(date));
+			}});
+
+			for (LocalDateTime time : DATE_TIMES) {
+				String expectedKey = String.join(delimiter, String.valueOf(date), IP);
+				String actualKey = partitioner.getPartitionId(createElement(time, IP));
+
+				assertEquals(expectedKey, actualKey);
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Negative tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testBadPartitionerWithEmptyPrefix() {
+		thrown.expect(IllegalArgumentException.class);
+		thrown.expect(containsCause(new IllegalArgumentException(
+			"Cannot create a RowDataFieldsKinesisPartitioner for a non-partitioned table")));
+
+		CatalogTable table = createTable(defaultTableOptions(), Collections.emptyList());
+		new RowDataFieldsKinesisPartitioner(table);
+	}
+
+	@Test
+	public void testBadPartitionerWithDuplicateFieldNamesInPrefix() {
+		thrown.expect(IllegalArgumentException.class);
+		thrown.expect(containsCause(new IllegalArgumentException(
+			"The sequence of partition keys cannot contain duplicates")));
+
+		CatalogTable table = createTable(defaultTableOptions(), Arrays.asList("ip", "ip"));
+		new RowDataFieldsKinesisPartitioner(table);
+	}
+
+	@Test
+	public void testBadPartitionerWithBadFieldFieldNamesInPrefix() {
+		thrown.expect(IllegalArgumentException.class);
+		thrown.expect(containsCause(new IllegalArgumentException(
+			"The following partition keys are not present in the table: abc")));
+
+		CatalogTable table = createTable(defaultTableOptions(), Arrays.asList("ip", "abc"));
+		new RowDataFieldsKinesisPartitioner(table);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Utilities
+	// --------------------------------------------------------------------------------------------
+
+	private RowData createElement(LocalDateTime time, String ip) {
+		return createElement(time, ip, "https://flink.apache.org/home");
+	}
+
+	private RowData createElement(LocalDateTime time, String ip, String route) {
+		GenericRowData element = new GenericRowData(TABLE_SCHEMA.getFieldCount());
+		element.setField(0, TimestampData.fromLocalDateTime(time));
+		element.setField(1, days(time));
+		element.setField(2, StringData.fromString(ip));
+		element.setField(3, StringData.fromString(route));
+		return element;
+	}
+
+	private int days(LocalDateTime time) {
+		return (int) ChronoUnit.DAYS.between(LocalDate.ofEpochDay(0), time);
+	}
+
+	private CatalogTable createTable(TableOptionsBuilder options, List<String> partitionKeys) {
+		return new CatalogTableImpl(TABLE_SCHEMA, partitionKeys, options.build(), TABLE_NAME);
+	}
+
+	private TableOptionsBuilder defaultTableOptions() {
+		String connector = KinesisDynamicTableFactory.IDENTIFIER;
+		String format = TestFormatFactory.IDENTIFIER;
+		return new TableOptionsBuilder(connector, format)
+			// default table options
+			.withTableOption(KinesisOptions.STREAM, TABLE_NAME)
+			.withTableOption("properties.aws.region", "us-west-2")
+			// default format options
+			.withFormatOption(TestFormatFactory.DELIMITER, ",");
+	}
+
+	/**
+	 * Repeat {@code n} times the string {@code s}.
+	 *
+	 * @param s The string to be repeated.
+	 * @param n The number of times to repeat the string.
+	 *
+	 * @return The repeated string.
+	 */
+	private static String repeat(String s, int n) {

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a23ca89bdf7d92515892df5ea5ac765f83e86856 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521) 
   * 6ffeeef6d919ed2691e41ba5130f98292ab8379f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 115d8d0f08bbdee1ca6becd32a0ca8698b486d72 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810) 
   * 8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908) 
   * 9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f1f444a6be029f1558052f93e2a3571aa4ab3613 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * f09502d6fa6f4f7122359956293194455decb303 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200) 
   * a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917) 
   * 88abeded29996fbeb101ec7e5498f1c33ff4ae67 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001) 
   * c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a23ca89bdf7d92515892df5ea5ac765f83e86856 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521) 
   * 6ffeeef6d919ed2691e41ba5130f98292ab8379f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535) 
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * c195c6d2c594e176fdbc038ee9672a90dd10ef80 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-721763948


   Let me answer the threads:
   
   ### Testing
   
   I was hoping we have some dummy Kinesis infrastructure already to test our connectors locally. If this is not the case, I guess we have to write a nightly end-to-end test for this purpose in a follow up issue.
   
   ### Property naming
   
   Let's discuss all metadata and properties keys real quick here before you implement them.
   
   ### Partitioning
   
   I agree. The JavaDoc is confusing in this case. I also had to read it again and was confused. I think the sentence refers to the example and should rather be `If only a subset of all partition keys (in our example a prefix part) get`. I cannot remember that we considered static partitions as only prefixes. Feel free to add another hotfix commit to fix this.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-721763948


   Let me answer the threads:
   
   ### Testing
   
   I was hoping we have some dummy Kinesis infrastructure already to test our connectors locally. If this is not the case, I guess we have to write a nightly end-to-end test for this purpose in a follow up issue.
   
   ### Property naming
   
   Let's discuss all metadata and properties keys real quick here before you implement them.
   
   ### Partitioning
   
   I agree. The JavaDoc is confusing in this case. I also had to read it again and was confused. I think the sentence refers to the example and should rather be `If only a subset of all partition keys (in our example a prefix part) get`. I cannot remember that we considered static partitions as only prefixes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517277900



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       > we should test the contents of mutable member variables 
   
   These are tested by the following `assertEquals(expectedSource, actualSource);` call, where `KinesisDynamicSource::equals` is
   
   ```java
   @Override
   public boolean equals(Object o) {
   	if (this == o) {
   		return true;
   	}
   	if (o == null || getClass() != o.getClass()) {
   		return false;
   	}
   	KinesisDynamicSource that = (KinesisDynamicSource) o;
   	return Objects.equals(producedDataType, that.producedDataType) &&
   		Objects.equals(requestedMetadataFields, that.requestedMetadataFields) &&
   		Objects.equals(physicalDataType, that.physicalDataType) &&
   		Objects.equals(stream, that.stream) &&
   		Objects.equals(consumerProperties, that.consumerProperties) &&
   		Objects.equals(decodingFormat, that.decodingFormat);
   }
   ```
   
   This goes somewhat against [the coding guideline](https://flink.apache.org/contributing/code-style-and-quality-java.html#equals--hashcode), but it is consistent with other connectors implementing ability interfaces (including `KafkaDynamicSource` and `KafkaDynamicSink`). 
   
   For me this is indicative of a broader issue with the ability interfaces and I suggest to open a separate JIRA to discuss this. IMHO calls of ability interface methods should return modified copies of the original source/sink node as opposed to mutating that node (this is compatible with the current design of these classes as immutable factories).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c9034134f3319c2ac5856d3e41c133148eb39e71 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409) 
   * a23ca89bdf7d92515892df5ea5ac765f83e86856 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723341214


   @twalthr for the next iteration please check the following two `fixup!` commits:
   
   - 96c368253be8bcd395ed05f5f99b3f07cbca4f52, which adds support for non-prefix static keys in `RowDataFieldsKinesisPartitioner`. Test coverage is added in
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticPrefix`,
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticSuffix`, and
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticInfix`.
   - c763dced7f725f78ddf9e1c0b83b29d7769b00a5, which reworks option names and mapping [as discussed in the GoogleDoc](https://docs.google.com/document/d/1Gchs3gK3nnWyodZMYWyK2q0APRQf1gofsTxKBzwzPLQ/edit?usp=sharing).
     - The `KinesisOptions#getConsumerProperties` implements the mapping of the properties for the Table source.
     - The `KinesisOptions#getProducerProperties` implements the mapping of the properties for the Table sink.
     - Test coverage is added to the happy path tests in `KinesisDynamicTableFactoryTest`.
   
   ### TODOs:
   
   - [ ] Create a follow-up JIRA in oder to add a `KinesisTableITCase`.
   - [ ] Create a follow-up JIRA in order to add a `flink-sql-connector-kinesis` package. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a23ca89bdf7d92515892df5ea5ac765f83e86856 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521) 
   * 6ffeeef6d919ed2691e41ba5130f98292ab8379f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535) 
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-716748372


   I added the a new documentation section under `./docs/dev/table/connectors/kinesis.md`.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 88ae6048ac2efeb04ebc076a93da3667c013ccf6 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191) 
   * f1f444a6be029f1558052f93e2a3571aa4ab3613 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723341214


   @twalthr for the next iteration please check the following two `fixup!` commits:
   
   - 96c368253be8bcd395ed05f5f99b3f07cbca4f52, which adds support for non-prefix static keys in `RowDataFieldsKinesisPartitioner`. Test coverage is added in
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticPrefix`,
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticSuffix`, and
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticInfix`.
   - c763dced7f725f78ddf9e1c0b83b29d7769b00a5, which reworks option names and mapping [as discussed in the GoogleDoc](https://docs.google.com/document/d/1Gchs3gK3nnWyodZMYWyK2q0APRQf1gofsTxKBzwzPLQ/edit?usp=sharing).
     - `KinesisOptions#getConsumerProperties` maps the table option keys to a `Properties` instance aligned with the key values expected by a `FlinkKinesisConsumer`.
     - `KinesisOptions#getProducerProperties` maps the table option keys to a `Properties` instance aligned with the key values expected by a `FlinkKinesisProducer`.
   
   ### TODOs:
   
   - [ ] Create a follow-up JIRA in oder to add a `KinesisTableITCase`.
   - [ ] Create a follow-up JIRA in order to add a `flink-sql-connector-kinesis` package. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517305280



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FixedKinesisPartitioner.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Objects;
+
+/**
+ * A partitioner ensuring that each internal Flink partition ends up in the same Kinesis partition.
+ *
+ * <p>This is achieved by using the index of the producer task as a {@code PartitionKey}.</p>
+ */
+@PublicEvolving
+public class FixedKinesisPartitioner<T> extends KinesisPartitioner<T> {

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517303510



##########
File path: flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestFormatFactory.java
##########
@@ -140,8 +142,28 @@ public int hashCode() {
 		}
 	}
 
+	/**
+	 * {@link DeserializationSchema} for testing.
+	 */
+	public static class DeserializationSchemaMock implements DeserializationSchema<RowData> {

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723341214


   @twalthr for the next iteration please check the following two `fixup!` commits:
   
   - 96c368253be8bcd395ed05f5f99b3f07cbca4f52, which adds support for non-prefix static keys in `RowDataFieldsKinesisPartitioner`. Test coverage is added in
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticPrefix`,
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticSuffix`, and
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticInfix`.
   - c763dced7f725f78ddf9e1c0b83b29d7769b00a5, which reworks option names and mapping [as discussed in the GoogleDoc](https://docs.google.com/document/d/1Gchs3gK3nnWyodZMYWyK2q0APRQf1gofsTxKBzwzPLQ/edit?usp=sharing).
     - `KinesisOptions#getConsumerProperties` maps the table option keys to a `Properties` instance aligned with the key values expected by a `FlinkKinesisConsumer`.
     - `KinesisOptions#getProducerProperties` maps the table option keys to a `Properties` instance aligned with the key values expected by a `FlinkKinesisProducer`.
   
   ### TODOs:
   
   - [x] Created FLINK-20042 in oder to add a `KinesisTableITCase`.
   - [x] Created FLINK-20043 in order to add a `flink-sql-connector-kinesis` package. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016) 
   * f09502d6fa6f4f7122359956293194455decb303 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200) 
   * a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c9034134f3319c2ac5856d3e41c133148eb39e71 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409) 
   * a23ca89bdf7d92515892df5ea5ac765f83e86856 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517283527



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);
+		actualSource.applyReadableMetadata(metadataKeys, producedDataType);
+
+		// verify that the constructed DynamicTableSink is as expected

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908) 
   * 9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a5ce7bc8a6c533b701598debf84e6dc2287f152a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326) 
   * c9034134f3319c2ac5856d3e41c133148eb39e71 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 115d8d0f08bbdee1ca6becd32a0ca8698b486d72 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810) 
   * 8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908) 
   * 9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r516020250



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataKinesisDeserializationSchema.java
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A {@link KinesisDeserializationSchema} adaptor for {@link RowData} records that delegates
+ * physical data deserialization to an inner {@link DeserializationSchema} and appends requested
+ * metadata to the end of the deserialized {@link RowData} record.
+ */
+@Internal
+public class RowDataKinesisDeserializationSchema implements KinesisDeserializationSchema<RowData> {
+
+	/** Internal type for enumerating available metadata. */
+	protected enum Metadata {
+		ApproximateArrivalTimestamp(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull()),
+		SequenceNumber(DataTypes.VARCHAR(128).notNull()),
+		ShardId(DataTypes.VARCHAR(128).notNull());
+
+		private final DataType dataType;
+
+		Metadata(DataType dataType) {
+			this.dataType = dataType;
+		}
+
+		public DataType getDataType() {
+			return this.dataType;
+		}
+	}
+
+	private static final long serialVersionUID = 5551095193778230749L;
+
+	/** A {@link DeserializationSchema} to deserialize the physical part of the row. */
+	private final DeserializationSchema<RowData> physicalDeserializer;
+
+	/** The type of the produced {@link RowData} records (physical data with appended metadata]. */
+	private final TypeInformation<RowData> producedTypeInfo;
+
+	/** Metadata fields to be appended to the physical {@link RowData} in the produced records. */
+	private final List<Metadata> requestedMetadataFields;
+
+	public RowDataKinesisDeserializationSchema(
+		DeserializationSchema<RowData> physicalDeserializer,
+		TypeInformation<RowData> producedTypeInfo,
+		List<Metadata> requestedMetadataFields) {
+		this.physicalDeserializer = Preconditions.checkNotNull(physicalDeserializer);
+		this.producedTypeInfo = Preconditions.checkNotNull(producedTypeInfo);
+		this.requestedMetadataFields = Preconditions.checkNotNull(requestedMetadataFields);
+	}
+
+	@Override
+	public void open(DeserializationSchema.InitializationContext context) throws Exception {
+		physicalDeserializer.open(context);
+	}
+
+	@Override
+	public RowData deserialize(
+		byte[] recordValue,
+		String partitionKey,
+		String seqNum,
+		long approxArrivalTimestamp,
+		String stream,
+		String shardId) throws IOException {
+
+		RowData physicalRow = physicalDeserializer.deserialize(recordValue);
+		GenericRowData metadataRow = new GenericRowData(requestedMetadataFields.size());
+
+		for (int i = 0; i < metadataRow.getArity(); i++) {
+			Metadata metadataField = requestedMetadataFields.get(i);
+			if (metadataField == Metadata.ApproximateArrivalTimestamp) {
+				metadataRow.setField(i, TimestampData.fromEpochMillis(approxArrivalTimestamp));
+			} else if (metadataField == Metadata.SequenceNumber) {
+				metadataRow.setField(i, StringData.fromString(seqNum));
+			} else if (metadataField == Metadata.ShardId) {
+				metadataRow.setField(i, StringData.fromString(shardId));
+			} else {
+				String msg = String.format("Unsupported metadata key %s", metadataField);
+				throw new RuntimeException(msg); // should never happen
+			}
+		}
+
+		JoinedRowData joinedRowData = new JoinedRowData(physicalRow, metadataRow);

Review comment:
       *Note*: I had to move `JoinedRowData` from `flink-table-runtime-blink` to `flink-table-common` to make this work. IMHO this is a more robust way of extending a record with metadata [than the current implementation in the `DynamicKafkaDeserializationSchema`](https://github.com/apache/flink/blob/d21f5d94474bd14eaa2d3c0dc386b81190533a4b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java#L115-L131).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9226",
       "triggerID" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 42df702f984d75eb8ec74bc1a48613b20c0bb3c5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9226) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517277900



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       > we should test the contents of mutable member variables 
   
   These are tested by the following `assertEquals(expectedSource, actualSource);` call, where `KinesisDynamicSource::equals` is
   
   ```java
   @Override
   public boolean equals(Object o) {
   	if (this == o) {
   		return true;
   	}
   	if (o == null || getClass() != o.getClass()) {
   		return false;
   	}
   	KinesisDynamicSource that = (KinesisDynamicSource) o;
   	return Objects.equals(producedDataType, that.producedDataType) &&
   		Objects.equals(requestedMetadataFields, that.requestedMetadataFields) &&
   		Objects.equals(physicalDataType, that.physicalDataType) &&
   		Objects.equals(stream, that.stream) &&
   		Objects.equals(consumerProperties, that.consumerProperties) &&
   		Objects.equals(decodingFormat, that.decodingFormat);
   }
   ```
   
   This goes somewhat against [the coding guideline](https://flink.apache.org/contributing/code-style-and-quality-java.html#equals--hashcode), but it is consistent with other connectors implementing ability interfaces (including `KafkaDynamicSource` and `KafkaDynamicSink`). For me this is indicative of a broader issue with the ability interfaces. IMHO calls of ability interface methods should return modified copies of the original source/sink node as opposed to mutating that node (this is compatible with the current design of these classes as immutable factories).  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517349077



##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,334 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Available Metadata
+------------------
+
+The following metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Column Name</th>
+      <th class="text-center" style="width: 45%">Column Type</th>
+      <th class="text-center" style="width: 35%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-ApproximateArrivalTimestamp">ApproximateArrivalTimestamp</a></code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIMEZONE NOT NULL</code></td>
+      <td>The approximate time that the record was inserted into the stream.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Shard.html#Streams-Type-Shard-ShardId">ShardId</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the shard within the stream from which the record was read.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-SequenceNumber">SequenceNumber</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the record within its shard.</td>
+    </tr>
+    </tbody>
+</table>
+
+The extended `CREATE TABLE` example demonstrates the syntax for exposing these metadata columns:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3),
+ arrival_time TIMESTAMP(3) METADATA FROM 'ApproximateArrivalTimestamp' VIRTUAL,
+ shard_id VARCHAR(128) NOT NULL METADATA FROM 'ShardId' VIRTUAL,
+ sequence_number VARCHAR(128) NOT NULL METADATA FROM 'SequenceNumber' VIRTUAL
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Connector Options
+-----------------
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Option</th>
+      <th class="text-center" style="width: 8%">Required</th>
+      <th class="text-center" style="width: 7%">Default</th>
+      <th class="text-center" style="width: 10%">Type</th>
+      <th class="text-center" style="width: 50%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><h5>connector</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Specify what connector to use. For Kinesis use <code>'kinesis'</code>.</td>
+    </tr>
+    <tr>
+      <td><h5>stream</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Name of the Kinesis data stream backing this table.</td>
+    </tr>
+    <tr>
+      <td><h5>format</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The format used to deserialize and serialize Kinesis data stream records. See <a href="#data-type-mapping">Data Type Mapping</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">random or row-based</td>
+      <td>String</td>
+      <td>Optional output partitioning from Flink's partitions into Kinesis shards. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner.field.delimiter</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">|</td>
+      <td>String</td>
+      <td>Optional field delimiter for a fields-based partitioner derived from a PARTITION BY clause. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.region</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS region where the stream is defined. Either this or <code>properties.aws.endpoint</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.endpoint</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS endpoint for Kinesis (derived from the AWS region setting if not set). Either this or <code>properties.aws.region</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.credentials.provider</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">AUTO</td>
+      <td>String</td>
+      <td>A credentials provider to use when authenticating against the Kinesis endpoint. See <a href="#authentication">Authentication</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.initpos</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">LATEST</td>
+      <td>String</td>
+      <td>Initial position to be used when reading from the table. See <a href="#start-reading-position">Start Reading Position</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.recordpublisher</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">POLLING</td>
+      <td>String</td>
+      <td>The `RecordPublisher` type to use for sources. See <a href="#enhanced-fan-out">Enhanced Fan-Out</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.*</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>
+      Other properties to pass to the FlinkKinesisConsumer or FlinkKinesisProducer constructors. 
+      See the constants defined in 
+      <ul>
+        <li><code>AWSConfigConstants</code>,</li> 
+        <li><code>ConsumerConfigConstants</code>, and</li> 
+        <li><code>ProducerConfigConstants</code></li>
+      </ul>
+      for detailed information on the available suffixes.
+      </td>
+    </tr>
+    </tbody>
+</table>
+
+Features
+--------
+
+### Authorization
+
+Make sure to [create an appropriate IAM policy](https://docs.aws.amazon.com/streams/latest/dev/controlling-access.html) to allow reading from / writing to the Kinesis data streams.
+
+### Authentication
+
+Depending on your deployment you would choose a different Credentials Provider to allow access to Kinesis.
+By default, the `AUTO` Credentials Provider is used.
+If the access key ID and secret key are set in the deployment configuration, this results in using the `BASIC` provider.  
+
+A specific [AWSCredentialsProvider](https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/index.html?com/amazonaws/auth/AWSCredentialsProvider.html) can be **optionally** set using the `properties.aws.credentials.provider` setting.
+Supported values are:
+
+* `AUTO` - Use the default AWS Credentials Provider chain that searches for credentials in the following order: `ENV_VARS`, `SYS_PROPS`, `WEB_IDENTITY_TOKEN`, `PROFILE`, and EC2/ECS credentials provider.
+* `BASIC` - Use access key ID and secret key supplied as configuration. 
+* `ENV_VAR` - Use `AWS_ACCESS_KEY_ID` & `AWS_SECRET_ACCESS_KEY` environment variables.
+* `SYS_PROP` - Use Java system properties `aws.accessKeyId` and `aws.secretKey`.
+* `PROFILE` - Use an AWS credentials profile to create the AWS credentials.
+* `ASSUME_ROLE` - Create AWS credentials by assuming a role. The credentials for assuming the role must be supplied.
+* `WEB_IDENTITY_TOKEN` - Create AWS credentials by assuming a role using Web Identity Token. 
+
+### Start Reading Position
+
+You can configure table sources to start reading a table-backing Kinesis data stream from a specific position through the `properties.flink.stream.initpos` option.

Review comment:
       I adopted the `properties.*` prefix-based based on a previous discussion (#2228 and #4473). The consensus from the discussion in these PRs was to:
   
   - Stick with naming closed to the KDS documentation so people familiar with official Kinesis terminology can find they way around more quickly [see this comment](https://github.com/apache/flink/pull/2228).
   - Make sure that users can directly pass-through options to the underlying KCL `KinesisProducerConfiguration` class [see this comment](https://github.com/apache/flink/pull/4473#issuecomment-320855718).
   
   This refers at components exposed at a lower layer (`DataSet` & `DataStream` APIs) and creates tension with the desire to keep property key names unified across different connectors at the Table API & SQL layers.
   
   I suggest to do the following:
   
   - Keys that are related to flink-level configuration will be exposed with table-api names and re-mapped internally by the factory components (so for example `properties.flink.stream.initpos` will be exposed at the SQL layer as `scan.stream.initpos`.
   - Keys that are passed directly to `KinesisProducerConfiguration` will be exposed with `kinesis.*` prefix.

##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,334 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Available Metadata
+------------------
+
+The following metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Column Name</th>
+      <th class="text-center" style="width: 45%">Column Type</th>
+      <th class="text-center" style="width: 35%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-ApproximateArrivalTimestamp">ApproximateArrivalTimestamp</a></code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIMEZONE NOT NULL</code></td>
+      <td>The approximate time that the record was inserted into the stream.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Shard.html#Streams-Type-Shard-ShardId">ShardId</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the shard within the stream from which the record was read.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-SequenceNumber">SequenceNumber</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the record within its shard.</td>
+    </tr>
+    </tbody>
+</table>
+
+The extended `CREATE TABLE` example demonstrates the syntax for exposing these metadata columns:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3),
+ arrival_time TIMESTAMP(3) METADATA FROM 'ApproximateArrivalTimestamp' VIRTUAL,

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9226",
       "triggerID" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fae6a040af833f0ef251dd4094617aa601fe081",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8fae6a040af833f0ef251dd4094617aa601fe081",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 42df702f984d75eb8ec74bc1a48613b20c0bb3c5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9226) 
   * 8fae6a040af833f0ef251dd4094617aa601fe081 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a23ca89bdf7d92515892df5ea5ac765f83e86856 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521) 
   * 6ffeeef6d919ed2691e41ba5130f98292ab8379f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535) 
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * c195c6d2c594e176fdbc038ee9672a90dd10ef80 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723341214


   @twalthr for the next iteration please check the following two `fixup!` commits:
   
   - 96c368253be8bcd395ed05f5f99b3f07cbca4f52, which adds support for non-prefix static keys in `RowDataFieldsKinesisPartitioner`. Test coverage is added in
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticPrefix`,
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticSuffix`, and
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticInfix`.
   - c763dced7f725f78ddf9e1c0b83b29d7769b00a5, which reworks option names and mapping [as discussed in the GoogleDoc](https://docs.google.com/document/d/1Gchs3gK3nnWyodZMYWyK2q0APRQf1gofsTxKBzwzPLQ/edit?usp=sharing).
     - `KinesisOptions#getConsumerProperties` maps the table option keys to a `Properties` instance aligned with the key values expected by a `FlinkKinesisConsumer`.
     - `KinesisOptions#getProducerProperties` maps the table option keys to a `Properties` instance aligned with the key values expected by a `FlinkKinesisProducer`.
   
   ### TODOs:
   
   - [x] Created FLINK-20042 in oder to add a `KinesisTableITCase`.
   - [ ] Create a follow-up JIRA in order to add a `flink-sql-connector-kinesis` package. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723490011


   Merging...


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517514237



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataKinesisDeserializationSchema.java
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A {@link KinesisDeserializationSchema} adaptor for {@link RowData} records that delegates
+ * physical data deserialization to an inner {@link DeserializationSchema} and appends requested
+ * metadata to the end of the deserialized {@link RowData} record.
+ */
+@Internal
+public class RowDataKinesisDeserializationSchema implements KinesisDeserializationSchema<RowData> {
+
+	/** Internal type for enumerating available metadata. */
+	protected enum Metadata {
+		ApproximateArrivalTimestamp(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull()),
+		SequenceNumber(DataTypes.VARCHAR(128).notNull()),
+		ShardId(DataTypes.VARCHAR(128).notNull());
+
+		private final DataType dataType;
+
+		Metadata(DataType dataType) {
+			this.dataType = dataType;
+		}
+
+		public DataType getDataType() {
+			return this.dataType;
+		}
+	}
+
+	private static final long serialVersionUID = 5551095193778230749L;
+
+	/** A {@link DeserializationSchema} to deserialize the physical part of the row. */
+	private final DeserializationSchema<RowData> physicalDeserializer;
+
+	/** The type of the produced {@link RowData} records (physical data with appended metadata]. */
+	private final TypeInformation<RowData> producedTypeInfo;
+
+	/** Metadata fields to be appended to the physical {@link RowData} in the produced records. */
+	private final List<Metadata> requestedMetadataFields;
+
+	public RowDataKinesisDeserializationSchema(
+		DeserializationSchema<RowData> physicalDeserializer,
+		TypeInformation<RowData> producedTypeInfo,
+		List<Metadata> requestedMetadataFields) {
+		this.physicalDeserializer = Preconditions.checkNotNull(physicalDeserializer);
+		this.producedTypeInfo = Preconditions.checkNotNull(producedTypeInfo);
+		this.requestedMetadataFields = Preconditions.checkNotNull(requestedMetadataFields);
+	}
+
+	@Override
+	public void open(DeserializationSchema.InitializationContext context) throws Exception {
+		physicalDeserializer.open(context);
+	}
+
+	@Override
+	public RowData deserialize(
+		byte[] recordValue,
+		String partitionKey,
+		String seqNum,
+		long approxArrivalTimestamp,
+		String stream,
+		String shardId) throws IOException {
+
+		RowData physicalRow = physicalDeserializer.deserialize(recordValue);
+		GenericRowData metadataRow = new GenericRowData(requestedMetadataFields.size());
+
+		for (int i = 0; i < metadataRow.getArity(); i++) {
+			Metadata metadataField = requestedMetadataFields.get(i);
+			if (metadataField == Metadata.ApproximateArrivalTimestamp) {
+				metadataRow.setField(i, TimestampData.fromEpochMillis(approxArrivalTimestamp));
+			} else if (metadataField == Metadata.SequenceNumber) {
+				metadataRow.setField(i, StringData.fromString(seqNum));
+			} else if (metadataField == Metadata.ShardId) {
+				metadataRow.setField(i, StringData.fromString(shardId));
+			} else {
+				String msg = String.format("Unsupported metadata key %s", metadataField);
+				throw new RuntimeException(msg); // should never happen
+			}
+		}
+
+		JoinedRowData joinedRowData = new JoinedRowData(physicalRow, metadataRow);

Review comment:
       OK, I added the relocation to a `utils` in a separate commit (different than the one that does the Maven module relocation). This will make it easier to revert the change and maintain backward-compatibility of the fully-qualified name without breaking the `RowDataKinesisDeserializationSchema` if needed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517277900



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       > we should test the contents of mutable member variables 
   
   These are tested by the following `assertEquals(expectedSource, actualSource);` call, where `KinesisDynamicSource::equals` is
   
   ```java
   @Override
   public boolean equals(Object o) {
   	if (this == o) {
   		return true;
   	}
   	if (o == null || getClass() != o.getClass()) {
   		return false;
   	}
   	KinesisDynamicSource that = (KinesisDynamicSource) o;
   	return Objects.equals(producedDataType, that.producedDataType) &&
   		Objects.equals(requestedMetadataFields, that.requestedMetadataFields) &&
   		Objects.equals(physicalDataType, that.physicalDataType) &&
   		Objects.equals(stream, that.stream) &&
   		Objects.equals(consumerProperties, that.consumerProperties) &&
   		Objects.equals(decodingFormat, that.decodingFormat);
   }
   ```
   
   This goes somewhat against [the coding guideline](https://flink.apache.org/contributing/code-style-and-quality-java.html#equals--hashcode), but it is consistent with other connectors implementing ability interfaces (including `KafkaDynamicSource`). For me this is indicative of a broader issue with the ability interfaces. IMHO calls of ability interface methods should return modified copies of the original source/sink node as opposed to mutating that node (this is compatible with the current design of these classes as immutable factories).  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f1f444a6be029f1558052f93e2a3571aa4ab3613 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205) 
   * a5ce7bc8a6c533b701598debf84e6dc2287f152a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-721757666


   Thanks for the thorough review @twalthr.
   
   I addressed most of the individual comments (additional feedback is required where the comment is not marked as *resolved*).
   
   Regarding your high-level comments:
   
   ## Testing
   
   I have a working end-to-end test written as an example in `flink-examples-table`. I can rewrite it as an `~ITCase`, however it is not clear to me what kind of workflow I should use to create / delete the backing Kinesis streams.
   
   Do we have an AWS account that we can use for ITCases? If not, I suggest to add a working `~ITCase` as `@ignored` and enable it when we decide how to resolve the issue of resource provisioning and access management for the AWS resources required for the integration tests.
   
   ## Property naming
   
   I will implement the suggestion proposed [at the bottom of this comment](https://github.com/apache/flink/pull/13770#discussion_r517349077). I will try to fix that today.
   
   ## Partitioning
   
   I was following the advise from the `SupportsPartitioning` JavaDoc:
   
   > If only a subset of all partition keys (a prefix part) get a value assigned in the PARTITION clause, the operation is considered as an "insertion into a dynamic partition".
   
   The term *prefix part* is quite confusing in this case, a better naming is *static part* or *fixed part*. I will try to fix that today.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917) 
   * 88abeded29996fbeb101ec7e5498f1c33ff4ae67 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517312314



##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,334 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Available Metadata
+------------------
+
+The following metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Column Name</th>
+      <th class="text-center" style="width: 45%">Column Type</th>
+      <th class="text-center" style="width: 35%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-ApproximateArrivalTimestamp">ApproximateArrivalTimestamp</a></code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIMEZONE NOT NULL</code></td>
+      <td>The approximate time that the record was inserted into the stream.</td>

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715305041


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 81adac78eaea29496bd8145811cd909a5d7123ef (Fri Oct 23 12:14:15 UTC 2020)
   
   **Warnings:**
    * **1 pom.xml files were touched**: Check for build and licensing issues.
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a5ce7bc8a6c533b701598debf84e6dc2287f152a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] whummer edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
whummer edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-721820441


   Many thanks @aalexandrov for this implementation - really exciting to see this feature being added to Flink! 🚀 
   
   > I was hoping we have some dummy Kinesis infrastructure already to test our connectors locally. 
   
   @twalthr Perhaps [LocalStack](https://github.com/localstack/localstack) could help out here? It supports an emulated version of Kinesis (and other AWS APIs) running locally, packaged as a Docker image. (If anything should be missing for testing this, we'd be very happy to add any required functionality in LocalStack, as needed.) Thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r518812707



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/JoinedRowData.java
##########
@@ -16,9 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.data;
+package org.apache.flink.table.data.utils;

Review comment:
       I don't have strong opinion here but it is marked `@Internal` and is not a core `RowData` class. It has not even a proper hashCode/equals. To me this is rather a utility class.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a5ce7bc8a6c533b701598debf84e6dc2287f152a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326) 
   * c9034134f3319c2ac5856d3e41c133148eb39e71 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723440957


   > Can you rebase this PR after you have addressed my comments?
   
   Done. When merging, note that:
   
   - The two `fixup!` commits need to be squashed.
   - You need to decide whether to include or exclude [the `JoinedRowData` package relocation commit]( https://github.com/apache/flink/pull/13770/commits/42df702f984d75eb8ec74bc1a48613b20c0bb3c5).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209) 
   * 42df702f984d75eb8ec74bc1a48613b20c0bb3c5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 88ae6048ac2efeb04ebc076a93da3667c013ccf6 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 88ae6048ac2efeb04ebc076a93da3667c013ccf6 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191) 
   * f1f444a6be029f1558052f93e2a3571aa4ab3613 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9226",
       "triggerID" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fae6a040af833f0ef251dd4094617aa601fe081",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9259",
       "triggerID" : "8fae6a040af833f0ef251dd4094617aa601fe081",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 42df702f984d75eb8ec74bc1a48613b20c0bb3c5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9226) 
   * 8fae6a040af833f0ef251dd4094617aa601fe081 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9259) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r519159919



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -224,13 +251,17 @@ public boolean equals(Object o) {
 		}
 		final RowDataFieldsKinesisPartitioner that = (RowDataFieldsKinesisPartitioner) o;
 		return Objects.equals(this.fieldNames, that.fieldNames) &&
-			Arrays.equals(this.fixedPrefix, that.fixedPrefix);
+			Objects.equals(this.staticFields, that.staticFields) &&
+			Objects.equals(this.keyBufferStaticPrefixLength, that.keyBufferStaticPrefixLength) &&
+			Objects.equals(this.fieldNamesStaticPrefixLength, that.fieldNamesStaticPrefixLength);
 	}
 
 	@Override
 	public int hashCode() {
 		return Objects.hash(
 			fieldNames,
-			Arrays.hashCode(fixedPrefix));
+			staticFields,
+			this.keyBufferStaticPrefixLength,
+			this.keyBufferStaticPrefixLength);

Review comment:
       nit: we can remove the `this`

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisOptions.java
##########
@@ -44,26 +49,54 @@
 		private KinesisOptions() {
 		}
 
-		// --------------------------------------------------------------------------------------------
+		// -----------------------------------------------------------------------------------------
 		// Kinesis specific options
-		// --------------------------------------------------------------------------------------------
+		// -----------------------------------------------------------------------------------------
 
 		/**
-		 * Prefix for properties delegated to the
+		 * Prefix for properties defined in
+		 * {@link org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants}
+		 * that are delegated to
 		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer} and
 		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer}.
 		 */
-		public static final String PROPERTIES_PREFIX = "properties.";
+		public static final String AWS_PROPERTIES_PREFIX = "aws.";
+
+		/**
+		 * Prefix for properties defined in
+		 * {@link org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants}
+		 * that are delegated to
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer}.
+		 */
+		public static final String CONSUMER_PREFIX = "source.";

Review comment:
       I'm very sorry. Somehow I missed this in the Google docs document. Instead of `source` we should use `scan` according to FLIP-122. Because we have two types of sources, we should make it clear that all these properties related to a table scan and not a lookup source.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -224,13 +251,17 @@ public boolean equals(Object o) {
 		}
 		final RowDataFieldsKinesisPartitioner that = (RowDataFieldsKinesisPartitioner) o;
 		return Objects.equals(this.fieldNames, that.fieldNames) &&
-			Arrays.equals(this.fixedPrefix, that.fixedPrefix);
+			Objects.equals(this.staticFields, that.staticFields) &&
+			Objects.equals(this.keyBufferStaticPrefixLength, that.keyBufferStaticPrefixLength) &&
+			Objects.equals(this.fieldNamesStaticPrefixLength, that.fieldNamesStaticPrefixLength);
 	}
 
 	@Override
 	public int hashCode() {
 		return Objects.hash(
 			fieldNames,
-			Arrays.hashCode(fixedPrefix));
+			staticFields,
+			this.keyBufferStaticPrefixLength,
+			this.keyBufferStaticPrefixLength);

Review comment:
       should be `fieldNamesStaticPrefixLength`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9226",
       "triggerID" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8fae6a040af833f0ef251dd4094617aa601fe081",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9259",
       "triggerID" : "8fae6a040af833f0ef251dd4094617aa601fe081",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 8fae6a040af833f0ef251dd4094617aa601fe081 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9259) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r519167157



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisOptions.java
##########
@@ -44,26 +49,54 @@
 		private KinesisOptions() {
 		}
 
-		// --------------------------------------------------------------------------------------------
+		// -----------------------------------------------------------------------------------------
 		// Kinesis specific options
-		// --------------------------------------------------------------------------------------------
+		// -----------------------------------------------------------------------------------------
 
 		/**
-		 * Prefix for properties delegated to the
+		 * Prefix for properties defined in
+		 * {@link org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants}
+		 * that are delegated to
 		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer} and
 		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer}.
 		 */
-		public static final String PROPERTIES_PREFIX = "properties.";
+		public static final String AWS_PROPERTIES_PREFIX = "aws.";
+
+		/**
+		 * Prefix for properties defined in
+		 * {@link org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants}
+		 * that are delegated to
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer}.
+		 */
+		public static final String CONSUMER_PREFIX = "source.";

Review comment:
       No problem.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517378013



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       We had this discussions before. Personally, I was also more in favor of immutable interfaces. But after lots of online and offline discussion we decided for mutability by considering `DynamicTableSource` and `DynamicTableSink` as another level of factories or builders that the planner uses to get the runtime implementation in the end. 
   
   Regarding this test, the `KinesisDynamicSource::equals` is correct but we should rather test the contents of the member variables instead of comparing the result of performing the same computation twice.
   
   Btw I wouldn't say that the current implementation goes against the coding guidelines. The mutable attributes must be included in the hashCode/equals and it might be the case that the planner is using them.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r512612401



##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,270 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Connector Options
+----------------
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Option</th>
+      <th class="text-center" style="width: 8%">Required</th>
+      <th class="text-center" style="width: 7%">Default</th>
+      <th class="text-center" style="width: 10%">Type</th>
+      <th class="text-center" style="width: 50%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><h5>connector</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Specify what connector to use. For Kinesis use <code>'kinesis'</code>.</td>
+    </tr>
+    <tr>
+      <td><h5>stream</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Name of the Kinesis data stream backing this table.</td>
+    </tr>
+    <tr>
+      <td><h5>format</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The format used to deserialize and serialize Kinesis data stream records. See <a href="#data-type-mapping">Data Type Mapping</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">random or row-based</td>
+      <td>String</td>
+      <td>Optional output partitioning from Flink's partitions into Kinesis shards. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.region</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS region where the stream is defined. Either this or <code>properties.aws.endpoint</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.endpoint</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS endpoint for Kinesis (derived from the AWS region setting if not set). Either this or <code>properties.aws.region</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.credentials.provider</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">AUTO</td>
+      <td>String</td>
+      <td>A credentials provider to use when authenticating against the Kinesis endpoint. See <a href="#authentication">Authentication</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.initpos</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">LATEST</td>
+      <td>String</td>
+      <td>Initial position to be used when reading from the table. See <a href="#start-reading-position">Start Reading Position</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.recordpublisher</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">POLLING</td>
+      <td>String</td>
+      <td>The `RecordPublisher` type to use for sources. See <a href="#enhanced-fan-out">Enhanced Fan-Out</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.*</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>
+      Other properties to pass to the FlinkKinesisConsumer or FlinkKinesisProducer constructors. 
+      See the constants defined in 
+      <ul>
+        <li><code>AWSConfigConstants</code>,</li> 
+        <li><code>ConsumerConfigConstants</code>, and</li> 
+        <li><code>ProducerConfigConstants</code></li>
+      </ul>
+      for detailed information on the available options.
+      </td>
+    </tr>
+    </tbody>
+</table>
+
+Features
+----------------
+
+### Authorization
+
+Make sure to [create an appropriate IAM policy](https://docs.aws.amazon.com/streams/latest/dev/controlling-access.html) to allow reading from / writing to the Kinesis data streams.
+
+### Authentication
+
+Depending on your deployment you would choose a different Credentials Provider to allow access to Kinesis.
+By default, the `AUTO` Credentials Provider is used.
+If the access key ID and secret key are set in the deployment configuration, this results in using the `BASIC` provider.  
+
+A specific Credentials Provider can be **optionally** set using the `properties.aws.credentials.provider` setting.
+Supported values are:
+
+* `AUTO` - Use the default AWS Credentials Provider chain that searches for credentials in the following order: `ENV_VARS`, `SYS_PROPS`, `WEB_IDENTITY_TOKEN`, `PROFILE`, and EC2/ECS credentials provider.
+* `BASIC` - Use access key ID and secret key supplied as configuration. 
+* `ENV_VAR` - Use `AWS_ACCESS_KEY_ID` & `AWS_SECRET_ACCESS_KEY` environment variables.
+* `SYS_PROP` - Use Java system properties `aws.accessKeyId` and `aws.secretKey`.
+* `PROFILE` - Use an AWS credentials profile to create the AWS credentials.
+* `ASSUME_ROLE` - Create AWS credentials by assuming a role. The credentials for assuming the role must be supplied.
+* `WEB_IDENTITY_TOKEN` - Create AWS credentials by assuming a role using Web Identity Token. 
+
+### Start Reading Position
+
+You can configure the `FlinkKinesisConsumer` to start reading the table-backing Kinesis data stream from a specific position through the `properties.flink.stream.initpos` option.
+Available values are:
+
+* `LATEST`: read all shards starting from the latest record.
+* `TRIM_HORIZON`: read all shards starting from the earliest record possible (data may be trimmed by Kinesis depending on the current retention settings of the backing stream).
+* `AT_TIMESTAMP`: read all shards starting from a specified timestamp. The timestamp must also be specified in the configuration
+properties by providing a value for `properties.flink.stream.initpos.timestamp`, in one of the following date patterns:
+   * A non-negative double value representing the number of seconds that has elapsed since the Unix epoch (for example, `1459799926.480`).
+   * A user defined pattern, which is a valid pattern for the `SimpleDateFormat` provided by `properties.flink.stream.initpos.timestamp.format`.
+     If a user-defined format is not supplied, the default pattern will be `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`.
+     For example, timestamp value is `2016-04-04` and user-defined foramt is `yyyy-MM-dd`, or timestamp value is `2016-04-04T19:58:46.480-00:00` and a user-defined format is not provided.
+
+### Sink Partitioning
+
+Kinesis data streams consist of one or more shards, and the `sink.partitioner` option allows you to control how records written into a multi-shard Kinesis-backe table will be partitioned between its shards.
+Valid values are:
+
+* `fixed`: Kinesis `PartitionKey` values derived from the Flink subtask index, so each Flink partition ends up in at most one Kinesis partition (assuming that no re-sharding takes place at runtime).
+* `random`: Kinesis `PartitionKey`</code>` values are assigned randomly.

Review comment:
       I'm not aware of any. I usually just render the Flink docs and see if they are fine.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517309867



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataKinesisDeserializationSchema.java
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A {@link KinesisDeserializationSchema} adaptor for {@link RowData} records that delegates
+ * physical data deserialization to an inner {@link DeserializationSchema} and appends requested
+ * metadata to the end of the deserialized {@link RowData} record.
+ */
+@Internal
+public class RowDataKinesisDeserializationSchema implements KinesisDeserializationSchema<RowData> {
+
+	/** Internal type for enumerating available metadata. */
+	protected enum Metadata {
+		ApproximateArrivalTimestamp(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull()),
+		SequenceNumber(DataTypes.VARCHAR(128).notNull()),
+		ShardId(DataTypes.VARCHAR(128).notNull());
+
+		private final DataType dataType;
+
+		Metadata(DataType dataType) {
+			this.dataType = dataType;
+		}
+
+		public DataType getDataType() {
+			return this.dataType;
+		}
+	}
+
+	private static final long serialVersionUID = 5551095193778230749L;
+
+	/** A {@link DeserializationSchema} to deserialize the physical part of the row. */
+	private final DeserializationSchema<RowData> physicalDeserializer;
+
+	/** The type of the produced {@link RowData} records (physical data with appended metadata]. */
+	private final TypeInformation<RowData> producedTypeInfo;
+
+	/** Metadata fields to be appended to the physical {@link RowData} in the produced records. */
+	private final List<Metadata> requestedMetadataFields;
+
+	public RowDataKinesisDeserializationSchema(
+		DeserializationSchema<RowData> physicalDeserializer,
+		TypeInformation<RowData> producedTypeInfo,
+		List<Metadata> requestedMetadataFields) {
+		this.physicalDeserializer = Preconditions.checkNotNull(physicalDeserializer);
+		this.producedTypeInfo = Preconditions.checkNotNull(producedTypeInfo);
+		this.requestedMetadataFields = Preconditions.checkNotNull(requestedMetadataFields);
+	}
+
+	@Override
+	public void open(DeserializationSchema.InitializationContext context) throws Exception {
+		physicalDeserializer.open(context);
+	}
+
+	@Override
+	public RowData deserialize(
+		byte[] recordValue,
+		String partitionKey,
+		String seqNum,
+		long approxArrivalTimestamp,
+		String stream,
+		String shardId) throws IOException {
+
+		RowData physicalRow = physicalDeserializer.deserialize(recordValue);
+		GenericRowData metadataRow = new GenericRowData(requestedMetadataFields.size());
+
+		for (int i = 0; i < metadataRow.getArity(); i++) {
+			Metadata metadataField = requestedMetadataFields.get(i);
+			if (metadataField == Metadata.ApproximateArrivalTimestamp) {
+				metadataRow.setField(i, TimestampData.fromEpochMillis(approxArrivalTimestamp));
+			} else if (metadataField == Metadata.SequenceNumber) {
+				metadataRow.setField(i, StringData.fromString(seqNum));
+			} else if (metadataField == Metadata.ShardId) {
+				metadataRow.setField(i, StringData.fromString(shardId));
+			} else {
+				String msg = String.format("Unsupported metadata key %s", metadataField);
+				throw new RuntimeException(msg); // should never happen
+			}
+		}
+
+		JoinedRowData joinedRowData = new JoinedRowData(physicalRow, metadataRow);

Review comment:
       I factored this change in a separate commit.
   
   I was hesitant to change the package name, though, as I am not sure whether this will be backwards-compatible. What will happen with savepoints that refer to serialized JoinedRowData instances after an 1.11.x → 1.12.0 upgrade? If you think this is safe I will change the package name.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517277900



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       These are tested by the following `assertEquals(expectedSource, actualSource);` call, where `KinesisDynamicSource::equals` is
   
   ```
   @Override
   public boolean equals(Object o) {
   	if (this == o) {
   		return true;
   	}
   	if (o == null || getClass() != o.getClass()) {
   		return false;
   	}
   	KinesisDynamicSource that = (KinesisDynamicSource) o;
   	return Objects.equals(producedDataType, that.producedDataType) &&
   		Objects.equals(requestedMetadataFields, that.requestedMetadataFields) &&
   		Objects.equals(physicalDataType, that.physicalDataType) &&
   		Objects.equals(stream, that.stream) &&
   		Objects.equals(consumerProperties, that.consumerProperties) &&
   		Objects.equals(decodingFormat, that.decodingFormat);
   }
   ```
   
   This goes somewhat [against the coding guideline](https://flink.apache.org/contributing/code-style-and-quality-java.html#equals--hashcode), but it is consistent with all other connectors (including `KafkaDynamicSource`). For me this is indicative of a broader issue with the ability interfaces. IMHO calls of ability interface methods should return modified copies of the original source/sink node as opposed to mutating that node (this is compatible with the current design of these classes as immutable factories).  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517365917



##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,334 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Available Metadata
+------------------
+
+The following metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Column Name</th>
+      <th class="text-center" style="width: 45%">Column Type</th>
+      <th class="text-center" style="width: 35%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-ApproximateArrivalTimestamp">ApproximateArrivalTimestamp</a></code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIMEZONE NOT NULL</code></td>
+      <td>The approximate time that the record was inserted into the stream.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Shard.html#Streams-Type-Shard-ShardId">ShardId</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the shard within the stream from which the record was read.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-SequenceNumber">SequenceNumber</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the record within its shard.</td>
+    </tr>
+    </tbody>
+</table>
+
+The extended `CREATE TABLE` example demonstrates the syntax for exposing these metadata columns:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3),
+ arrival_time TIMESTAMP(3) METADATA FROM 'ApproximateArrivalTimestamp' VIRTUAL,
+ shard_id VARCHAR(128) NOT NULL METADATA FROM 'ShardId' VIRTUAL,
+ sequence_number VARCHAR(128) NOT NULL METADATA FROM 'SequenceNumber' VIRTUAL
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Connector Options
+-----------------
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Option</th>
+      <th class="text-center" style="width: 8%">Required</th>
+      <th class="text-center" style="width: 7%">Default</th>
+      <th class="text-center" style="width: 10%">Type</th>
+      <th class="text-center" style="width: 50%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><h5>connector</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Specify what connector to use. For Kinesis use <code>'kinesis'</code>.</td>
+    </tr>
+    <tr>
+      <td><h5>stream</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Name of the Kinesis data stream backing this table.</td>
+    </tr>
+    <tr>
+      <td><h5>format</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The format used to deserialize and serialize Kinesis data stream records. See <a href="#data-type-mapping">Data Type Mapping</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">random or row-based</td>
+      <td>String</td>
+      <td>Optional output partitioning from Flink's partitions into Kinesis shards. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner.field.delimiter</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">|</td>
+      <td>String</td>
+      <td>Optional field delimiter for a fields-based partitioner derived from a PARTITION BY clause. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.region</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS region where the stream is defined. Either this or <code>properties.aws.endpoint</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.endpoint</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS endpoint for Kinesis (derived from the AWS region setting if not set). Either this or <code>properties.aws.region</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.credentials.provider</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">AUTO</td>
+      <td>String</td>
+      <td>A credentials provider to use when authenticating against the Kinesis endpoint. See <a href="#authentication">Authentication</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.initpos</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">LATEST</td>
+      <td>String</td>
+      <td>Initial position to be used when reading from the table. See <a href="#start-reading-position">Start Reading Position</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.recordpublisher</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">POLLING</td>
+      <td>String</td>
+      <td>The `RecordPublisher` type to use for sources. See <a href="#enhanced-fan-out">Enhanced Fan-Out</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.*</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>
+      Other properties to pass to the FlinkKinesisConsumer or FlinkKinesisProducer constructors. 
+      See the constants defined in 
+      <ul>
+        <li><code>AWSConfigConstants</code>,</li> 
+        <li><code>ConsumerConfigConstants</code>, and</li> 
+        <li><code>ProducerConfigConstants</code></li>
+      </ul>
+      for detailed information on the available suffixes.
+      </td>
+    </tr>
+    </tbody>
+</table>
+
+Features
+--------
+
+### Authorization
+
+Make sure to [create an appropriate IAM policy](https://docs.aws.amazon.com/streams/latest/dev/controlling-access.html) to allow reading from / writing to the Kinesis data streams.
+
+### Authentication
+
+Depending on your deployment you would choose a different Credentials Provider to allow access to Kinesis.
+By default, the `AUTO` Credentials Provider is used.
+If the access key ID and secret key are set in the deployment configuration, this results in using the `BASIC` provider.  
+
+A specific [AWSCredentialsProvider](https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/index.html?com/amazonaws/auth/AWSCredentialsProvider.html) can be **optionally** set using the `properties.aws.credentials.provider` setting.
+Supported values are:
+
+* `AUTO` - Use the default AWS Credentials Provider chain that searches for credentials in the following order: `ENV_VARS`, `SYS_PROPS`, `WEB_IDENTITY_TOKEN`, `PROFILE`, and EC2/ECS credentials provider.
+* `BASIC` - Use access key ID and secret key supplied as configuration. 
+* `ENV_VAR` - Use `AWS_ACCESS_KEY_ID` & `AWS_SECRET_ACCESS_KEY` environment variables.
+* `SYS_PROP` - Use Java system properties `aws.accessKeyId` and `aws.secretKey`.
+* `PROFILE` - Use an AWS credentials profile to create the AWS credentials.
+* `ASSUME_ROLE` - Create AWS credentials by assuming a role. The credentials for assuming the role must be supplied.
+* `WEB_IDENTITY_TOKEN` - Create AWS credentials by assuming a role using Web Identity Token. 
+
+### Start Reading Position
+
+You can configure table sources to start reading a table-backing Kinesis data stream from a specific position through the `properties.flink.stream.initpos` option.

Review comment:
       Thanks for the detailed investigation. Naming things was never easy :D But a tradeoff between SQL property naming convention and previous discussions sounds good to me. Can you quickly list all property keys here?
   
   For example, for Kafka we did not prefix all properties with `kafka.` but we kept the top-level, most frequently used properties as short as possible (e.g. `topic`). The properties that are directly passed through to the lower layer are represented under `properties.*`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-721898904


   Following up on the main topics:
   
   ### Testing
   
   > If this is not the case, I guess we have to write a nightly end-to-end test for this purpose in a follow up issue.
   
   The only end-to-end examples that I can see in the `test` package are located under `org.apache.flink.streaming.connectors.kinesis.examples`, but instead of implementing an `ITCase` these classes define self-contained examples in a `main` method. I will look into @whummer's suggestion and follow-up with the end-to-end testing in a separate JIRA (if possible, the aim will be to use a mock infrastructure). 
   
   ### Property naming
   
   > Let's discuss all metadata and properties keys real quick here before you implement them.
   
   I will sketch a proposal in a separate comment.
   
   ### Partitioning
   
   > Feel free to add another hotfix commit to fix this.
   
   I did that.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * c195c6d2c594e176fdbc038ee9672a90dd10ef80 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr closed pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr closed pull request #13770:
URL: https://github.com/apache/flink/pull/13770


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 88ae6048ac2efeb04ebc076a93da3667c013ccf6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517515634



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       > Regarding this test, the KinesisDynamicSource::equals is correct but we should rather test the contents of the member variables instead of comparing the result of performing the same computation twice.
   
   I see what you mean now, I will fix that.
   
   > Btw I wouldn't say that the current implementation goes against the coding guidelines
   
   I was referring to this part of the guidelines  
   
   > When the methods take mutable fields into account, you often have a design issue.
   
   and the possible design issue with the mutable factories.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517277900



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       > we should test the contents of mutable member variables 
   
   These are tested by the following `assertEquals(expectedSource, actualSource);` call, where `KinesisDynamicSource::equals` is
   
   ```java
   @Override
   public boolean equals(Object o) {
   	if (this == o) {
   		return true;
   	}
   	if (o == null || getClass() != o.getClass()) {
   		return false;
   	}
   	KinesisDynamicSource that = (KinesisDynamicSource) o;
   	return Objects.equals(producedDataType, that.producedDataType) &&
   		Objects.equals(requestedMetadataFields, that.requestedMetadataFields) &&
   		Objects.equals(physicalDataType, that.physicalDataType) &&
   		Objects.equals(stream, that.stream) &&
   		Objects.equals(consumerProperties, that.consumerProperties) &&
   		Objects.equals(decodingFormat, that.decodingFormat);
   }
   ```
   
   This goes somewhat against [the coding guideline](https://flink.apache.org/contributing/code-style-and-quality-java.html#equals--hashcode), but it is consistent with all other connectors (including `KafkaDynamicSource`). For me this is indicative of a broader issue with the ability interfaces. IMHO calls of ability interface methods should return modified copies of the original source/sink node as opposed to mutating that node (this is compatible with the current design of these classes as immutable factories).  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9226",
       "triggerID" : "42df702f984d75eb8ec74bc1a48613b20c0bb3c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209) 
   * 42df702f984d75eb8ec74bc1a48613b20c0bb3c5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9226) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r518802767



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/JoinedRowData.java
##########
@@ -16,9 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.data;
+package org.apache.flink.table.data.utils;

Review comment:
       I would suggest to move this class to `org.apache.flink.table.data`, because I think it is a kind of `RowData` which is similar to `GenericRowData`, not a utility. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r518709552



##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,334 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Available Metadata
+------------------
+
+The following metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Column Name</th>
+      <th class="text-center" style="width: 45%">Column Type</th>
+      <th class="text-center" style="width: 35%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-ApproximateArrivalTimestamp">ApproximateArrivalTimestamp</a></code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIMEZONE NOT NULL</code></td>
+      <td>The approximate time that the record was inserted into the stream.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Shard.html#Streams-Type-Shard-ShardId">ShardId</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the shard within the stream from which the record was read.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-SequenceNumber">SequenceNumber</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the record within its shard.</td>
+    </tr>
+    </tbody>
+</table>
+
+The extended `CREATE TABLE` example demonstrates the syntax for exposing these metadata columns:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3),
+ arrival_time TIMESTAMP(3) METADATA FROM 'ApproximateArrivalTimestamp' VIRTUAL,
+ shard_id VARCHAR(128) NOT NULL METADATA FROM 'ShardId' VIRTUAL,
+ sequence_number VARCHAR(128) NOT NULL METADATA FROM 'SequenceNumber' VIRTUAL
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Connector Options
+-----------------
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Option</th>
+      <th class="text-center" style="width: 8%">Required</th>
+      <th class="text-center" style="width: 7%">Default</th>
+      <th class="text-center" style="width: 10%">Type</th>
+      <th class="text-center" style="width: 50%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><h5>connector</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Specify what connector to use. For Kinesis use <code>'kinesis'</code>.</td>
+    </tr>
+    <tr>
+      <td><h5>stream</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Name of the Kinesis data stream backing this table.</td>
+    </tr>
+    <tr>
+      <td><h5>format</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The format used to deserialize and serialize Kinesis data stream records. See <a href="#data-type-mapping">Data Type Mapping</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">random or row-based</td>
+      <td>String</td>
+      <td>Optional output partitioning from Flink's partitions into Kinesis shards. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner.field.delimiter</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">|</td>
+      <td>String</td>
+      <td>Optional field delimiter for a fields-based partitioner derived from a PARTITION BY clause. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.region</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS region where the stream is defined. Either this or <code>properties.aws.endpoint</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.endpoint</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS endpoint for Kinesis (derived from the AWS region setting if not set). Either this or <code>properties.aws.region</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.credentials.provider</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">AUTO</td>
+      <td>String</td>
+      <td>A credentials provider to use when authenticating against the Kinesis endpoint. See <a href="#authentication">Authentication</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.initpos</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">LATEST</td>
+      <td>String</td>
+      <td>Initial position to be used when reading from the table. See <a href="#start-reading-position">Start Reading Position</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.recordpublisher</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">POLLING</td>
+      <td>String</td>
+      <td>The `RecordPublisher` type to use for sources. See <a href="#enhanced-fan-out">Enhanced Fan-Out</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.*</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>
+      Other properties to pass to the FlinkKinesisConsumer or FlinkKinesisProducer constructors. 
+      See the constants defined in 
+      <ul>
+        <li><code>AWSConfigConstants</code>,</li> 
+        <li><code>ConsumerConfigConstants</code>, and</li> 
+        <li><code>ProducerConfigConstants</code></li>
+      </ul>
+      for detailed information on the available suffixes.
+      </td>
+    </tr>
+    </tbody>
+</table>
+
+Features
+--------
+
+### Authorization
+
+Make sure to [create an appropriate IAM policy](https://docs.aws.amazon.com/streams/latest/dev/controlling-access.html) to allow reading from / writing to the Kinesis data streams.
+
+### Authentication
+
+Depending on your deployment you would choose a different Credentials Provider to allow access to Kinesis.
+By default, the `AUTO` Credentials Provider is used.
+If the access key ID and secret key are set in the deployment configuration, this results in using the `BASIC` provider.  
+
+A specific [AWSCredentialsProvider](https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/index.html?com/amazonaws/auth/AWSCredentialsProvider.html) can be **optionally** set using the `properties.aws.credentials.provider` setting.
+Supported values are:
+
+* `AUTO` - Use the default AWS Credentials Provider chain that searches for credentials in the following order: `ENV_VARS`, `SYS_PROPS`, `WEB_IDENTITY_TOKEN`, `PROFILE`, and EC2/ECS credentials provider.
+* `BASIC` - Use access key ID and secret key supplied as configuration. 
+* `ENV_VAR` - Use `AWS_ACCESS_KEY_ID` & `AWS_SECRET_ACCESS_KEY` environment variables.
+* `SYS_PROP` - Use Java system properties `aws.accessKeyId` and `aws.secretKey`.
+* `PROFILE` - Use an AWS credentials profile to create the AWS credentials.
+* `ASSUME_ROLE` - Create AWS credentials by assuming a role. The credentials for assuming the role must be supplied.
+* `WEB_IDENTITY_TOKEN` - Create AWS credentials by assuming a role using Web Identity Token. 
+
+### Start Reading Position
+
+You can configure table sources to start reading a table-backing Kinesis data stream from a specific position through the `properties.flink.stream.initpos` option.

Review comment:
       OK, I suggest to use the following options:
   
   
   <table class="table table-bordered">
       <thead>
       <tr>
         <th class="text-left" style="width: 25%">Option</th>
         <th class="text-center" style="width: 8%">Required</th>
         <th class="text-center" style="width: 7%">Default</th>
         <th class="text-center" style="width: 10%">Type</th>
         <th class="text-center" style="width: 50%">Description</th>
       </tr>
       </thead>
       <tbody>
       <tr>
         <td><h5>connector</h5></td>
         <td>required</td>
         <td style="word-wrap: break-word;">(none)</td>
         <td>String</td>
         <td>Specify what connector to use. For Kinesis use <code>'kinesis'</code>.</td>
       </tr>
       <tr>
         <td><h5>stream</h5></td>
         <td>required</td>
         <td style="word-wrap: break-word;">(none)</td>
         <td>String</td>
         <td>Name of the Kinesis data stream backing this table.</td>
       </tr>
       <tr>
         <td><h5>format</h5></td>
         <td>required</td>
         <td style="word-wrap: break-word;">(none)</td>
         <td>String</td>
         <td>The format used to deserialize and serialize Kinesis data stream records. See <a href="#data-type-mapping">Data Type Mapping</a> for details.</td>
       </tr>
       <tr>
         <td><h5>sink.partitioner</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">random or row-based</td>
         <td>String</td>
         <td>Optional output partitioning from Flink's partitions into Kinesis shards. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
       </tr>
       <tr>
         <td><h5>sink.partitioner.field.delimiter</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">|</td>
         <td>String</td>
         <td>Optional field delimiter for a fields-based partitioner derived from a PARTITION BY clause. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
       </tr>
       <tr>
         <td><h5>aws.region</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">(none)</td>
         <td>String</td>
         <td>The AWS region where the stream is defined. Either this or <code>aws.endpoint</code> are required.</td>
       </tr>
       <tr>
         <td><h5>aws.endpoint</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">(none)</td>
         <td>String</td>
         <td>The AWS endpoint for Kinesis (derived from the AWS region setting if not set). Either this or <code>aws.region</code> are required.</td>
       </tr>
       <tr>
         <td><h5>aws.credentials.provider</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">AUTO</td>
         <td>String</td>
         <td>A credentials provider to use when authenticating against the Kinesis endpoint. See <a href="#authentication">Authentication</a> for details.</td>
       </tr>
       <tr>
         <td><h5>source.stream.recordpublisher</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">POLLING</td>
         <td>String</td>
         <td>The <code>RecordPublisher</code> type to use for sources. See <a href="#enhanced-fan-out">Enhanced Fan-Out</a> for details.</td>
       </tr>
       <tr>
         <td><h5>source.stream.efo.consumername</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">(none)</td>
         <td>String</td>
         <td>The name of the EFO consumer to register with KDS.</td>
       </tr>
       <tr>
         <td><h5>source.stream.efo.registration</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">LAZY</td>
         <td>String</td>
         <td>Determine how and when consumer de-/registration is performed (LAZY|EAGER|NONE).</td>
       </tr>
       <tr>
         <td><h5>source.stream.efo.consumerarn</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">(none)</td>
         <td>String</td>
         <td>The prefix of consumer ARN for a given stream.</td>
       </tr>
       <tr>
         <td><h5>source.stream.initpos</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">LATEST</td>
         <td>String</td>
         <td>Initial position to be used when reading from the table. See <a href="#start-reading-position">Start Reading Position</a> for details.</td>
       </tr>
       <tr>
         <td><h5>source.stream.initpos.timestamp</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">(none)</td>
         <td>String</td>
         <td>The initial timestamp to start reading Kinesis stream from (when AT_TIMESTAMP is set for STREAM_INITIAL_POSITION).</td>
       </tr>
       <tr>
         <td><h5>source.stream.initpos.timestamp.format</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">yyyy-MM-dd'T'HH:mm:ss.SSSXXX</td>
         <td>String</td>
         <td>The date format of initial timestamp to start reading Kinesis stream from (when AT_TIMESTAMP is set for STREAM_INITIAL_POSITION).</td>
       </tr>
       <tr>
         <td><h5>source.stream.describe.maxretries</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">50</td>
         <td>Integer</td>
         <td>The maximum number of <code>describeStream</code> attempts if we get a recoverable exception.</td>
       </tr>
       <tr>
         <td><h5>source.stream.describe.backoff.base</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">2000</td>
         <td>Long</td>
         <td>The base backoff time (in milliseconds) between each <code>describeStream</code> attempt (for consuming from DynamoDB streams).</td>
       </tr>
       <tr>
         <td><h5>source.stream.describe.backoff.max</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">5000</td>
         <td>Long</td>
         <td>The maximum backoff time (in milliseconds)  between each <code>describeStream</code> attempt (for consuming from DynamoDB streams).</td>
       </tr>
       <tr>
         <td><h5>source.stream.describe.backoff.expconst</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1.5</td>
         <td>Double</td>
         <td>The power constant for exponential backoff between each <code>describeStream</code> attempt (for consuming from DynamoDB streams).</td>
       </tr>
       <tr>
         <td><h5>source.list.shards.maxretries</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">10</td>
         <td>Integer</td>
         <td>The maximum number of <code>listShards</code> attempts if we get a recoverable exception.</td>
       </tr>
       <tr>
         <td><h5>source.list.shards.backoff.base</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1000</td>
         <td>Long</td>
         <td>The base backoff time (in milliseconds) between each <code>listShards</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.list.shards.backoff.max</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">5000</td>
         <td>Long</td>
         <td>The maximum backoff time (in milliseconds) between each <code>listShards</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.list.shards.backoff.expconst</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1.5</td>
         <td>Double</td>
         <td>The power constant for exponential backoff between each <code>listShards</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.stream.describestreamconsumer.maxretries</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">50</td>
         <td>Integer</td>
         <td>The maximum number of <code>describeStreamConsumer</code> attempts if we get a recoverable exception.</td>
       </tr>
       <tr>
         <td><h5>source.stream.describestreamconsumer.backoff.base</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">2000</td>
         <td>Long</td>
         <td>The base backoff time (in milliseconds) between each <code>describeStreamConsumer</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.stream.describestreamconsumer.backoff.max</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">5000</td>
         <td>Long</td>
         <td>The maximum backoff time (in milliseconds) between each <code>describeStreamConsumer</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.stream.describestreamconsumer.backoff.expconst</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1.5</td>
         <td>Double</td>
         <td>The power constant for exponential backoff between each <code>describeStreamConsumer</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.stream.registerstreamconsumer.maxretries</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">10</td>
         <td>Integer</td>
         <td>The maximum number of <code>registerStream</code> attempts if we get a recoverable exception.</td>
       </tr>
       <tr>
         <td><h5>source.stream.registerstreamconsumer.timeout</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">60</td>
         <td>Integer</td>
         <td>The maximum time in seconds to wait for a stream consumer to become active before giving up.</td>
       </tr>
       <tr>
         <td><h5>source.stream.registerstreamconsumer.backoff.base</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">500</td>
         <td>Long</td>
         <td>The base backoff time (in milliseconds) between each <code>registerStream</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.stream.registerstreamconsumer.backoff.max</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">2000</td>
         <td>Long</td>
         <td>The maximum backoff time (in milliseconds) between each <code>registerStream</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.stream.registerstreamconsumer.backoff.expconst</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1.5</td>
         <td>Double</td>
         <td>The power constant for exponential backoff between each <code>registerStream</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.stream.deregisterstreamconsumer.maxretries</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">10</td>
         <td>Integer</td>
         <td>The maximum number of <code>deregisterStream</code> attempts if we get a recoverable exception.</td>
       </tr>
       <tr>
         <td><h5>source.stream.deregisterstreamconsumer.timeout</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">60</td>
         <td>Integer</td>
         <td>The maximum time in seconds to wait for a stream consumer to deregister before giving up.</td>
       </tr>
       <tr>
         <td><h5>source.stream.deregisterstreamconsumer.backoff.base</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">500</td>
         <td>Long</td>
         <td>The base backoff time (in milliseconds) between each <code>deregisterStream</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.stream.deregisterstreamconsumer.backoff.max</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">2000</td>
         <td>Long</td>
         <td>The maximum backoff time (in milliseconds) between each <code>deregisterStream</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.stream.deregisterstreamconsumer.backoff.expconst</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1.5</td>
         <td>Double</td>
         <td>The power constant for exponential backoff between each <code>deregisterStream</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.shard.subscribetoshard.maxretries</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">10</td>
         <td>Integer</td>
         <td>The maximum number of <code>subscribeToShard</code> attempts if we get a recoverable exception.</td>
       </tr>
       <tr>
         <td><h5>source.shard.subscribetoshard.backoff.base</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1000</td>
         <td>Long</td>
         <td>The base backoff time (in milliseconds) between each <code>subscribeToShard</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.shard.subscribetoshard.backoff.max</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">2000</td>
         <td>Long</td>
         <td>The maximum backoff time (in milliseconds) between each <code>subscribeToShard</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.shard.subscribetoshard.backoff.expconst</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1.5</td>
         <td>Double</td>
         <td>The power constant for exponential backoff between each <code>subscribeToShard</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getrecords.maxrecordcount</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">10000</td>
         <td>Integer</td>
         <td>The maximum number of records to try to get each time we fetch records from a AWS Kinesis shard.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getrecords.maxretries</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">3</td>
         <td>Integer</td>
         <td>The maximum number of <code>getRecords</code> attempts if we get a recoverable exception.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getrecords.backoff.base</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">300</td>
         <td>Long</td>
         <td>The base backoff time (in milliseconds) between <code>getRecords</code> attempts if we get a ProvisionedThroughputExceededException.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getrecords.backoff.max</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1000</td>
         <td>Long</td>
         <td>The maximum backoff time (in milliseconds) between <code>getRecords</code> attempts if we get a ProvisionedThroughputExceededException.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getrecords.backoff.expconst</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1.5</td>
         <td>Double</td>
         <td>The power constant for exponential backoff between each <code>getRecords</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getrecords.intervalmillis</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">200</td>
         <td>Long</td>
         <td>The interval (in milliseconds) between each <code>getRecords</code> request to a AWS Kinesis shard in milliseconds.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getiterator.maxretries</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">3</td>
         <td>Integer</td>
         <td>The maximum number of <code>getShardIterator</code> attempts if we get ProvisionedThroughputExceededException.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getiterator.backoff.base</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">300</td>
         <td>Long</td>
         <td>The base backoff time (in milliseconds) between <code>getShardIterator</code> attempts if we get a ProvisionedThroughputExceededException.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getiterator.backoff.max</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1000</td>
         <td>Long</td>
         <td>The maximum backoff time (in milliseconds) between <code>getShardIterator</code> attempts if we get a ProvisionedThroughputExceededException.</td>
       </tr>
       <tr>
         <td><h5>source.shard.getiterator.backoff.expconst</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">1.5</td>
         <td>Double</td>
         <td>The power constant for exponential backoff between each <code>getShardIterator</code> attempt.</td>
       </tr>
       <tr>
         <td><h5>source.shard.discovery.intervalmillis</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">10000</td>
         <td>Integer</td>
         <td>The interval between each attempt to discover new shards.</td>
       </tr>
       <tr>
         <td><h5>source.shard.adaptivereads</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">false</td>
         <td>Boolean</td>
         <td>The config to turn on adaptive reads from a shard. See the <code>AdaptivePollingRecordPublisher</code> documentation for details.</td>
       </tr>
       <tr>
         <td><h5>source.shard.idle.interval</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">-1</td>
         <td>Long</td>
         <td>The interval (in milliseconds) after which to consider a shard idle for purposes of watermark generation. A positive value will allow the watermark to progress even when some shards don't receive new records.</td>
       </tr>
       <tr>
         <td><h5>source.watermark.sync.interval</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">30000</td>
         <td>Long</td>
         <td>The interval (in milliseconds) for periodically synchronizing the shared watermark state.</td>
       </tr>
       <tr>
         <td><h5>source.watermark.lookahead.millis</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">0</td>
         <td>Long</td>
         <td>The maximum delta (in milliseconds) allowed for the reader to advance ahead of the shared global watermark.</td>
       </tr>
       <tr>
         <td><h5>source.watermark.sync.queue.capacity</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">100</td>
         <td>Integer</td>
         <td>The maximum number of records that will be buffered before suspending consumption of a shard.</td>
       </tr>
       <tr>
         <td><h5>source.stream.efo.http-client.max-concurrency</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">10000</td>
         <td>Integer</td>
         <td>Maximum number of allowed concurrent requests for the EFO client.</td>
       </tr>
       <tr>
         <td><h5>sink.*</h5></td>
         <td>optional</td>
         <td style="word-wrap: break-word;">(none)</td>
         <td></td>
         <td>
           Sink options for the <code>KinesisProducer</code>. 
           Passed after stripping the <code>sink.*</code> to <a href="https://javadoc.io/static/com.amazonaws/amazon-kinesis-producer/0.14.0/com/amazonaws/services/kinesis/producer/KinesisProducerConfiguration.html#fromProperties-java.util.Properties-">"KinesisProducerConfiguration#fromProperties</a>.
           Suffix names must match the <a href="https://javadoc.io/static/com.amazonaws/amazon-kinesis-producer/0.14.0/com/amazonaws/services/kinesis/producer/KinesisProducerConfiguration.html">KinesisProducerConfiguration</a> getters minus the <code>get</code> prefix (for example, <code>CollectionMaxCount</code> or <code>AggregationMaxCount</code>).
           Note that some of the KCL defaults are overwritten by <code>KinesisConfigUtil</code>.
         </td>
       </tr>
       </tbody>
   </table>
   
   Instead of duplicating code in `KinesisOptions`, I suggest to use the following rules when constructing the `Properties` map passed downstream:
   
   * Option keys starting with `source.*` are replaced by the corresponding `flink.*` property. Available keys are listed in `ConsumerConfigConstants`.
   * Option keys starting with `aws.*` are kept intact. Available keys are listed in `AWSConfigConstants`.
   * Option keys starting with `sink.*` are stripped from the `sink.` prefix and passed directly to the `FlinkKinesisProducer` constructor (this is in line with the discussion in [FLINK-7367](https://issues.apache.org/jira/browse/FLINK-7367)).
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723048446


   @wuchong / @twalthr I added [a list of the suggested properties in a Google Doc](https://docs.google.com/document/d/1Gchs3gK3nnWyodZMYWyK2q0APRQf1gofsTxKBzwzPLQ/edit?usp=sharing). Please review and suggest edits using the "Suggesting Mode" from the top-right corner. I suggest to keep the mapping strategy at the top of the document as simple as possible.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723444013


   FYI I will try to tackle FLINK-20043 before the feature-freeze tonight (you can assign me to that issue). What should be the testing strategy for the generated fat jar?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917) 
   * 88abeded29996fbeb101ec7e5498f1c33ff4ae67 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517280149



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicSource.java
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+
+/**
+ * Kinesis-backed {@link ScanTableSource}.
+ */
+@Internal
+public class KinesisDynamicSource implements ScanTableSource, SupportsReadingMetadata {
+
+	/** List of read-only metadata fields that the source can provide upstream upon request. */
+	private static final Map<String, DataType> READABLE_METADATA =
+		new HashMap<String, DataType>() {{
+			for (Metadata metadata : Metadata.values()) {
+				put(metadata.name(), metadata.getDataType());
+			}
+		}};
+
+	// --------------------------------------------------------------------------------------------
+	// Mutable attributes
+	// --------------------------------------------------------------------------------------------
+
+	/** Data type that describes the final output of the source. */
+	private DataType producedDataType;
+
+	/** Metadata that is requested to be appended at the end of a physical source row. */
+	private List<Metadata> requestedMetadataFields;
+
+	// --------------------------------------------------------------------------------------------
+	// Scan format attributes
+	// --------------------------------------------------------------------------------------------
+
+	/** Data type to configure the format. */
+	private final DataType physicalDataType;
+
+	/** Scan format for decoding records from Kinesis. */
+	private final DecodingFormat<DeserializationSchema<RowData>> decodingFormat;
+
+	// --------------------------------------------------------------------------------------------
+	// Kinesis-specific attributes
+	// --------------------------------------------------------------------------------------------
+
+	/** The Kinesis stream to consume. */
+	private final String stream;
+
+	/** Properties for the Kinesis consumer. */
+	private final Properties consumerProperties;
+
+	public KinesisDynamicSource(
+		DataType physicalDataType,
+		String stream,
+		Properties consumerProperties,
+		DecodingFormat<DeserializationSchema<RowData>> decodingFormat) {
+		this(
+			physicalDataType,
+			stream,
+			consumerProperties,
+			decodingFormat,
+			null,
+			Collections.emptyList());
+	}
+
+	public KinesisDynamicSource(
+		DataType physicalDataType,
+		String stream,
+		Properties consumerProperties,
+		DecodingFormat<DeserializationSchema<RowData>> decodingFormat,
+		@Nullable DataType producedDataType,
+		List<Metadata> requestedMetadataFields) {
+
+		this.producedDataType = this.physicalDataType = Preconditions.checkNotNull(
+			physicalDataType,
+			"Produced data type must not be null.");
+		this.stream = Preconditions.checkNotNull(
+			stream,
+			"Stream must not be null.");
+		this.consumerProperties = Preconditions.checkNotNull(
+			consumerProperties,
+			"Properties must not be null.");
+		this.decodingFormat = Preconditions.checkNotNull(
+			decodingFormat,
+			"Decoding format must not be null.");
+		this.producedDataType = producedDataType;

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r512456345



##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,270 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Connector Options
+----------------
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Option</th>
+      <th class="text-center" style="width: 8%">Required</th>
+      <th class="text-center" style="width: 7%">Default</th>
+      <th class="text-center" style="width: 10%">Type</th>
+      <th class="text-center" style="width: 50%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><h5>connector</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Specify what connector to use. For Kinesis use <code>'kinesis'</code>.</td>
+    </tr>
+    <tr>
+      <td><h5>stream</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Name of the Kinesis data stream backing this table.</td>
+    </tr>
+    <tr>
+      <td><h5>format</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The format used to deserialize and serialize Kinesis data stream records. See <a href="#data-type-mapping">Data Type Mapping</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">random or row-based</td>
+      <td>String</td>
+      <td>Optional output partitioning from Flink's partitions into Kinesis shards. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.region</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS region where the stream is defined. Either this or <code>properties.aws.endpoint</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.endpoint</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS endpoint for Kinesis (derived from the AWS region setting if not set). Either this or <code>properties.aws.region</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.credentials.provider</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">AUTO</td>
+      <td>String</td>
+      <td>A credentials provider to use when authenticating against the Kinesis endpoint. See <a href="#authentication">Authentication</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.initpos</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">LATEST</td>
+      <td>String</td>
+      <td>Initial position to be used when reading from the table. See <a href="#start-reading-position">Start Reading Position</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.recordpublisher</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">POLLING</td>
+      <td>String</td>
+      <td>The `RecordPublisher` type to use for sources. See <a href="#enhanced-fan-out">Enhanced Fan-Out</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.*</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>
+      Other properties to pass to the FlinkKinesisConsumer or FlinkKinesisProducer constructors. 
+      See the constants defined in 
+      <ul>
+        <li><code>AWSConfigConstants</code>,</li> 
+        <li><code>ConsumerConfigConstants</code>, and</li> 
+        <li><code>ProducerConfigConstants</code></li>
+      </ul>
+      for detailed information on the available options.
+      </td>
+    </tr>
+    </tbody>
+</table>
+
+Features
+----------------
+
+### Authorization
+
+Make sure to [create an appropriate IAM policy](https://docs.aws.amazon.com/streams/latest/dev/controlling-access.html) to allow reading from / writing to the Kinesis data streams.
+
+### Authentication
+
+Depending on your deployment you would choose a different Credentials Provider to allow access to Kinesis.
+By default, the `AUTO` Credentials Provider is used.
+If the access key ID and secret key are set in the deployment configuration, this results in using the `BASIC` provider.  
+
+A specific Credentials Provider can be **optionally** set using the `properties.aws.credentials.provider` setting.
+Supported values are:
+
+* `AUTO` - Use the default AWS Credentials Provider chain that searches for credentials in the following order: `ENV_VARS`, `SYS_PROPS`, `WEB_IDENTITY_TOKEN`, `PROFILE`, and EC2/ECS credentials provider.
+* `BASIC` - Use access key ID and secret key supplied as configuration. 
+* `ENV_VAR` - Use `AWS_ACCESS_KEY_ID` & `AWS_SECRET_ACCESS_KEY` environment variables.
+* `SYS_PROP` - Use Java system properties `aws.accessKeyId` and `aws.secretKey`.
+* `PROFILE` - Use an AWS credentials profile to create the AWS credentials.
+* `ASSUME_ROLE` - Create AWS credentials by assuming a role. The credentials for assuming the role must be supplied.
+* `WEB_IDENTITY_TOKEN` - Create AWS credentials by assuming a role using Web Identity Token. 
+
+### Start Reading Position
+
+You can configure the `FlinkKinesisConsumer` to start reading the table-backing Kinesis data stream from a specific position through the `properties.flink.stream.initpos` option.
+Available values are:
+
+* `LATEST`: read all shards starting from the latest record.
+* `TRIM_HORIZON`: read all shards starting from the earliest record possible (data may be trimmed by Kinesis depending on the current retention settings of the backing stream).
+* `AT_TIMESTAMP`: read all shards starting from a specified timestamp. The timestamp must also be specified in the configuration
+properties by providing a value for `properties.flink.stream.initpos.timestamp`, in one of the following date patterns:
+   * A non-negative double value representing the number of seconds that has elapsed since the Unix epoch (for example, `1459799926.480`).
+   * A user defined pattern, which is a valid pattern for the `SimpleDateFormat` provided by `properties.flink.stream.initpos.timestamp.format`.
+     If a user-defined format is not supplied, the default pattern will be `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`.
+     For example, timestamp value is `2016-04-04` and user-defined foramt is `yyyy-MM-dd`, or timestamp value is `2016-04-04T19:58:46.480-00:00` and a user-defined format is not provided.
+
+### Sink Partitioning
+
+Kinesis data streams consist of one or more shards, and the `sink.partitioner` option allows you to control how records written into a multi-shard Kinesis-backe table will be partitioned between its shards.
+Valid values are:
+
+* `fixed`: Kinesis `PartitionKey` values derived from the Flink subtask index, so each Flink partition ends up in at most one Kinesis partition (assuming that no re-sharding takes place at runtime).
+* `random`: Kinesis `PartitionKey`</code>` values are assigned randomly.

Review comment:
       ```suggestion
   * `random`: Kinesis `PartitionKey` values are assigned randomly.
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517277900



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       > we should test the contents of mutable member variables 
   
   These are tested by the following `assertEquals(expectedSource, actualSource);` call, where `KinesisDynamicSource::equals` is
   
   ```java
   @Override
   public boolean equals(Object o) {
   	if (this == o) {
   		return true;
   	}
   	if (o == null || getClass() != o.getClass()) {
   		return false;
   	}
   	KinesisDynamicSource that = (KinesisDynamicSource) o;
   	return Objects.equals(producedDataType, that.producedDataType) &&
   		Objects.equals(requestedMetadataFields, that.requestedMetadataFields) &&
   		Objects.equals(physicalDataType, that.physicalDataType) &&
   		Objects.equals(stream, that.stream) &&
   		Objects.equals(consumerProperties, that.consumerProperties) &&
   		Objects.equals(decodingFormat, that.decodingFormat);
   }
   ```
   
   This goes somewhat [against the coding guideline](https://flink.apache.org/contributing/code-style-and-quality-java.html#equals--hashcode), but it is consistent with all other connectors (including `KafkaDynamicSource`). For me this is indicative of a broader issue with the ability interfaces. IMHO calls of ability interface methods should return modified copies of the original source/sink node as opposed to mutating that node (this is compatible with the current design of these classes as immutable factories).  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f1f444a6be029f1558052f93e2a3571aa4ab3613 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205) 
   * a5ce7bc8a6c533b701598debf84e6dc2287f152a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r512602408



##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,270 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Connector Options
+----------------
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Option</th>
+      <th class="text-center" style="width: 8%">Required</th>
+      <th class="text-center" style="width: 7%">Default</th>
+      <th class="text-center" style="width: 10%">Type</th>
+      <th class="text-center" style="width: 50%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><h5>connector</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Specify what connector to use. For Kinesis use <code>'kinesis'</code>.</td>
+    </tr>
+    <tr>
+      <td><h5>stream</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Name of the Kinesis data stream backing this table.</td>
+    </tr>
+    <tr>
+      <td><h5>format</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The format used to deserialize and serialize Kinesis data stream records. See <a href="#data-type-mapping">Data Type Mapping</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">random or row-based</td>
+      <td>String</td>
+      <td>Optional output partitioning from Flink's partitions into Kinesis shards. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.region</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS region where the stream is defined. Either this or <code>properties.aws.endpoint</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.endpoint</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS endpoint for Kinesis (derived from the AWS region setting if not set). Either this or <code>properties.aws.region</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.credentials.provider</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">AUTO</td>
+      <td>String</td>
+      <td>A credentials provider to use when authenticating against the Kinesis endpoint. See <a href="#authentication">Authentication</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.initpos</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">LATEST</td>
+      <td>String</td>
+      <td>Initial position to be used when reading from the table. See <a href="#start-reading-position">Start Reading Position</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.recordpublisher</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">POLLING</td>
+      <td>String</td>
+      <td>The `RecordPublisher` type to use for sources. See <a href="#enhanced-fan-out">Enhanced Fan-Out</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.*</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>
+      Other properties to pass to the FlinkKinesisConsumer or FlinkKinesisProducer constructors. 
+      See the constants defined in 
+      <ul>
+        <li><code>AWSConfigConstants</code>,</li> 
+        <li><code>ConsumerConfigConstants</code>, and</li> 
+        <li><code>ProducerConfigConstants</code></li>
+      </ul>
+      for detailed information on the available options.
+      </td>
+    </tr>
+    </tbody>
+</table>
+
+Features
+----------------
+
+### Authorization
+
+Make sure to [create an appropriate IAM policy](https://docs.aws.amazon.com/streams/latest/dev/controlling-access.html) to allow reading from / writing to the Kinesis data streams.
+
+### Authentication
+
+Depending on your deployment you would choose a different Credentials Provider to allow access to Kinesis.
+By default, the `AUTO` Credentials Provider is used.
+If the access key ID and secret key are set in the deployment configuration, this results in using the `BASIC` provider.  
+
+A specific Credentials Provider can be **optionally** set using the `properties.aws.credentials.provider` setting.
+Supported values are:
+
+* `AUTO` - Use the default AWS Credentials Provider chain that searches for credentials in the following order: `ENV_VARS`, `SYS_PROPS`, `WEB_IDENTITY_TOKEN`, `PROFILE`, and EC2/ECS credentials provider.
+* `BASIC` - Use access key ID and secret key supplied as configuration. 
+* `ENV_VAR` - Use `AWS_ACCESS_KEY_ID` & `AWS_SECRET_ACCESS_KEY` environment variables.
+* `SYS_PROP` - Use Java system properties `aws.accessKeyId` and `aws.secretKey`.
+* `PROFILE` - Use an AWS credentials profile to create the AWS credentials.
+* `ASSUME_ROLE` - Create AWS credentials by assuming a role. The credentials for assuming the role must be supplied.
+* `WEB_IDENTITY_TOKEN` - Create AWS credentials by assuming a role using Web Identity Token. 
+
+### Start Reading Position
+
+You can configure the `FlinkKinesisConsumer` to start reading the table-backing Kinesis data stream from a specific position through the `properties.flink.stream.initpos` option.
+Available values are:
+
+* `LATEST`: read all shards starting from the latest record.
+* `TRIM_HORIZON`: read all shards starting from the earliest record possible (data may be trimmed by Kinesis depending on the current retention settings of the backing stream).
+* `AT_TIMESTAMP`: read all shards starting from a specified timestamp. The timestamp must also be specified in the configuration
+properties by providing a value for `properties.flink.stream.initpos.timestamp`, in one of the following date patterns:
+   * A non-negative double value representing the number of seconds that has elapsed since the Unix epoch (for example, `1459799926.480`).
+   * A user defined pattern, which is a valid pattern for the `SimpleDateFormat` provided by `properties.flink.stream.initpos.timestamp.format`.
+     If a user-defined format is not supplied, the default pattern will be `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`.
+     For example, timestamp value is `2016-04-04` and user-defined foramt is `yyyy-MM-dd`, or timestamp value is `2016-04-04T19:58:46.480-00:00` and a user-defined format is not provided.
+
+### Sink Partitioning
+
+Kinesis data streams consist of one or more shards, and the `sink.partitioner` option allows you to control how records written into a multi-shard Kinesis-backe table will be partitioned between its shards.
+Valid values are:
+
+* `fixed`: Kinesis `PartitionKey` values derived from the Flink subtask index, so each Flink partition ends up in at most one Kinesis partition (assuming that no re-sharding takes place at runtime).
+* `random`: Kinesis `PartitionKey`</code>` values are assigned randomly.

Review comment:
       Done. Is there a syntax checker for the markdown files that I can run?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-721757666


   Thanks for the thorough review @twalthr.
   
   I addressed most of the individual comments (additional feedback is required where the comment is not marked as *resolved*).
   
   Regarding your high-level comments:
   
   ### Testing
   
   I have a working end-to-end test written as an example in `flink-examples-table`. I can rewrite it as an `~ITCase`, however it is not clear to me what kind of workflow I should use to create / delete the backing Kinesis streams.
   
   Do we have an AWS account that we can use for ITCases? If not, I suggest to add a working `~ITCase` as `@ignored` and enable it when we decide how to resolve the issue of resource provisioning and access management for the AWS resources required for the integration tests.
   
   ### Property naming
   
   I will implement the suggestion proposed [at the bottom of this comment](https://github.com/apache/flink/pull/13770#discussion_r517349077). I will try to fix that today.
   
   ### Partitioning
   
   I was following the advise from the `SupportsPartitioning` JavaDoc:
   
   > If only a subset of all partition keys (a prefix part) get a value assigned in the PARTITION clause, the operation is considered as an "insertion into a dynamic partition".
   
   The term *prefix part* is quite confusing in this case, a better naming is *static part* or *fixed part*. I will try to fix that today.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723135211


   > We should support all kinds of static partitions (not just prefix ones) for usability.
   
   I added a fixup commit that changes the behavior of the `RowDataFieldsKinesisPartitioner`.
   
   > Also we should limit the supported data types for partitions, because only a limited set of internal data structures has a good toString method. We had similar problems for the Elasticsearch key.
   
   What is the recommended way to identify this set in terms of source code?
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r516020250



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataKinesisDeserializationSchema.java
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A {@link KinesisDeserializationSchema} adaptor for {@link RowData} records that delegates
+ * physical data deserialization to an inner {@link DeserializationSchema} and appends requested
+ * metadata to the end of the deserialized {@link RowData} record.
+ */
+@Internal
+public class RowDataKinesisDeserializationSchema implements KinesisDeserializationSchema<RowData> {
+
+	/** Internal type for enumerating available metadata. */
+	protected enum Metadata {
+		ApproximateArrivalTimestamp(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull()),
+		SequenceNumber(DataTypes.VARCHAR(128).notNull()),
+		ShardId(DataTypes.VARCHAR(128).notNull());
+
+		private final DataType dataType;
+
+		Metadata(DataType dataType) {
+			this.dataType = dataType;
+		}
+
+		public DataType getDataType() {
+			return this.dataType;
+		}
+	}
+
+	private static final long serialVersionUID = 5551095193778230749L;
+
+	/** A {@link DeserializationSchema} to deserialize the physical part of the row. */
+	private final DeserializationSchema<RowData> physicalDeserializer;
+
+	/** The type of the produced {@link RowData} records (physical data with appended metadata]. */
+	private final TypeInformation<RowData> producedTypeInfo;
+
+	/** Metadata fields to be appended to the physical {@link RowData} in the produced records. */
+	private final List<Metadata> requestedMetadataFields;
+
+	public RowDataKinesisDeserializationSchema(
+		DeserializationSchema<RowData> physicalDeserializer,
+		TypeInformation<RowData> producedTypeInfo,
+		List<Metadata> requestedMetadataFields) {
+		this.physicalDeserializer = Preconditions.checkNotNull(physicalDeserializer);
+		this.producedTypeInfo = Preconditions.checkNotNull(producedTypeInfo);
+		this.requestedMetadataFields = Preconditions.checkNotNull(requestedMetadataFields);
+	}
+
+	@Override
+	public void open(DeserializationSchema.InitializationContext context) throws Exception {
+		physicalDeserializer.open(context);
+	}
+
+	@Override
+	public RowData deserialize(
+		byte[] recordValue,
+		String partitionKey,
+		String seqNum,
+		long approxArrivalTimestamp,
+		String stream,
+		String shardId) throws IOException {
+
+		RowData physicalRow = physicalDeserializer.deserialize(recordValue);
+		GenericRowData metadataRow = new GenericRowData(requestedMetadataFields.size());
+
+		for (int i = 0; i < metadataRow.getArity(); i++) {
+			Metadata metadataField = requestedMetadataFields.get(i);
+			if (metadataField == Metadata.ApproximateArrivalTimestamp) {
+				metadataRow.setField(i, TimestampData.fromEpochMillis(approxArrivalTimestamp));
+			} else if (metadataField == Metadata.SequenceNumber) {
+				metadataRow.setField(i, StringData.fromString(seqNum));
+			} else if (metadataField == Metadata.ShardId) {
+				metadataRow.setField(i, StringData.fromString(shardId));
+			} else {
+				String msg = String.format("Unsupported metadata key %s", metadataField);
+				throw new RuntimeException(msg); // should never happen
+			}
+		}
+
+		JoinedRowData joinedRowData = new JoinedRowData(physicalRow, metadataRow);

Review comment:
       *Note*: I had to move `JoinedRowData` from `flink-table-runtime-blink` to `flink-table-common` to make this work. IMHO this is a more robust way of extending a record with metadata [than the implementation currently seen in `DynamicKafkaDeserializationSchema`](https://github.com/apache/flink/blob/d21f5d94474bd14eaa2d3c0dc386b81190533a4b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java#L115-L131).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c9034134f3319c2ac5856d3e41c133148eb39e71 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517359453



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataKinesisDeserializationSchema.java
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A {@link KinesisDeserializationSchema} adaptor for {@link RowData} records that delegates
+ * physical data deserialization to an inner {@link DeserializationSchema} and appends requested
+ * metadata to the end of the deserialized {@link RowData} record.
+ */
+@Internal
+public class RowDataKinesisDeserializationSchema implements KinesisDeserializationSchema<RowData> {
+
+	/** Internal type for enumerating available metadata. */
+	protected enum Metadata {
+		ApproximateArrivalTimestamp(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull()),
+		SequenceNumber(DataTypes.VARCHAR(128).notNull()),
+		ShardId(DataTypes.VARCHAR(128).notNull());
+
+		private final DataType dataType;
+
+		Metadata(DataType dataType) {
+			this.dataType = dataType;
+		}
+
+		public DataType getDataType() {
+			return this.dataType;
+		}
+	}
+
+	private static final long serialVersionUID = 5551095193778230749L;
+
+	/** A {@link DeserializationSchema} to deserialize the physical part of the row. */
+	private final DeserializationSchema<RowData> physicalDeserializer;
+
+	/** The type of the produced {@link RowData} records (physical data with appended metadata]. */
+	private final TypeInformation<RowData> producedTypeInfo;
+
+	/** Metadata fields to be appended to the physical {@link RowData} in the produced records. */
+	private final List<Metadata> requestedMetadataFields;
+
+	public RowDataKinesisDeserializationSchema(
+		DeserializationSchema<RowData> physicalDeserializer,
+		TypeInformation<RowData> producedTypeInfo,
+		List<Metadata> requestedMetadataFields) {
+		this.physicalDeserializer = Preconditions.checkNotNull(physicalDeserializer);
+		this.producedTypeInfo = Preconditions.checkNotNull(producedTypeInfo);
+		this.requestedMetadataFields = Preconditions.checkNotNull(requestedMetadataFields);
+	}
+
+	@Override
+	public void open(DeserializationSchema.InitializationContext context) throws Exception {
+		physicalDeserializer.open(context);
+	}
+
+	@Override
+	public RowData deserialize(
+		byte[] recordValue,
+		String partitionKey,
+		String seqNum,
+		long approxArrivalTimestamp,
+		String stream,
+		String shardId) throws IOException {
+
+		RowData physicalRow = physicalDeserializer.deserialize(recordValue);
+		GenericRowData metadataRow = new GenericRowData(requestedMetadataFields.size());
+
+		for (int i = 0; i < metadataRow.getArity(); i++) {
+			Metadata metadataField = requestedMetadataFields.get(i);
+			if (metadataField == Metadata.ApproximateArrivalTimestamp) {
+				metadataRow.setField(i, TimestampData.fromEpochMillis(approxArrivalTimestamp));
+			} else if (metadataField == Metadata.SequenceNumber) {
+				metadataRow.setField(i, StringData.fromString(seqNum));
+			} else if (metadataField == Metadata.ShardId) {
+				metadataRow.setField(i, StringData.fromString(shardId));
+			} else {
+				String msg = String.format("Unsupported metadata key %s", metadataField);
+				throw new RuntimeException(msg); // should never happen
+			}
+		}
+
+		JoinedRowData joinedRowData = new JoinedRowData(physicalRow, metadataRow);

Review comment:
       We do not provide any savepoint guarantees for SQL yet. We try to not change savepoints within minor versions but also this is just best effort. So we can safely move this class to a better package.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * 88abeded29996fbeb101ec7e5498f1c33ff4ae67 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001) 
   * c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c9034134f3319c2ac5856d3e41c133148eb39e71 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409) 
   * a23ca89bdf7d92515892df5ea5ac765f83e86856 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521) 
   * 6ffeeef6d919ed2691e41ba5130f98292ab8379f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r516768180



##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,334 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Available Metadata
+------------------
+
+The following metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Column Name</th>
+      <th class="text-center" style="width: 45%">Column Type</th>
+      <th class="text-center" style="width: 35%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-ApproximateArrivalTimestamp">ApproximateArrivalTimestamp</a></code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIMEZONE NOT NULL</code></td>
+      <td>The approximate time that the record was inserted into the stream.</td>

Review comment:
       `The approximate time when the record was inserted into the stream.`

##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,334 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Available Metadata
+------------------
+
+The following metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Column Name</th>
+      <th class="text-center" style="width: 45%">Column Type</th>
+      <th class="text-center" style="width: 35%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-ApproximateArrivalTimestamp">ApproximateArrivalTimestamp</a></code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIMEZONE NOT NULL</code></td>
+      <td>The approximate time that the record was inserted into the stream.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Shard.html#Streams-Type-Shard-ShardId">ShardId</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the shard within the stream from which the record was read.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-SequenceNumber">SequenceNumber</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the record within its shard.</td>
+    </tr>
+    </tbody>
+</table>
+
+The extended `CREATE TABLE` example demonstrates the syntax for exposing these metadata columns:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3),
+ arrival_time TIMESTAMP(3) METADATA FROM 'ApproximateArrivalTimestamp' VIRTUAL,

Review comment:
       We would like to give users a consistent property and metadata naming. Please follow the guidelines in `org.apache.flink.table.factories.Factory` as mentioned in `SupportsReadingMetadata#listReadableMetadata`. So `shard-id` instead of `ShardId` and `approximate-arrival-timestamp` instead of `ApproximateArrivalTimestamp`. Btw should we shorten it to `arrival-timestamp` or just `timestamp` like in Kafka? We try to keep the keys as concise as possible lately.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactory.java
##########
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Factory for creating {@link KinesisDynamicSource} and {@link KinesisDynamicSink} instances.
+ */
+@Internal
+public class KinesisDynamicTableFactory implements
+	DynamicTableSourceFactory,
+	DynamicTableSinkFactory {
+	public static final String IDENTIFIER = "kinesis";
+
+	// --------------------------------------------------------------------------------------------
+	// DynamicTableSourceFactory
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public DynamicTableSource createDynamicTableSource(Context context) {
+		FactoryUtil.TableFactoryHelper helper = FactoryUtil
+			.createTableFactoryHelper(this, context);
+		Properties connectorProperties = KinesisOptions

Review comment:
       nit: this is in the middle of the `helper` calls, I would put it further down to where it is needed

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisOptions.java
##########
@@ -0,0 +1,186 @@
+	/*
+	 * Licensed to the Apache Software Foundation (ASF) under one
+	 * or more contributor license agreements.  See the NOTICE file
+	 * distributed with this work for additional information
+	 * regarding copyright ownership.  The ASF licenses this file
+	 * to you under the Apache License, Version 2.0 (the
+	 * "License"); you may not use this file except in compliance
+	 * with the License.  You may obtain a copy of the License at
+	 *
+	 *     http://www.apache.org/licenses/LICENSE-2.0
+	 *
+	 * Unless required by applicable law or agreed to in writing, software
+	 * distributed under the License is distributed on an "AS IS" BASIS,
+	 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+	 * See the License for the specific language governing permissions and
+	 * limitations under the License.
+	 */
+
+	package org.apache.flink.streaming.connectors.kinesis.table;
+
+	import org.apache.flink.annotation.Internal;
+	import org.apache.flink.configuration.ConfigOption;
+	import org.apache.flink.configuration.ConfigOptions;
+	import org.apache.flink.configuration.ReadableConfig;
+	import org.apache.flink.streaming.connectors.kinesis.FixedKinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+	import org.apache.flink.table.api.ValidationException;
+	import org.apache.flink.table.catalog.CatalogTable;
+	import org.apache.flink.table.data.RowData;
+	import org.apache.flink.util.FlinkException;
+	import org.apache.flink.util.InstantiationUtil;
+
+	import java.util.Map;
+	import java.util.Properties;
+
+	/**
+	 * Options for Kinesis tables supported by the {@code CREATE TABLE ... WITH ...} clause of the
+	 * Flink SQL dialect and the Flink Table API.
+	 */
+	@Internal
+	public class KinesisOptions {
+
+		private KinesisOptions() {
+		}
+
+		// --------------------------------------------------------------------------------------------
+		// Kinesis specific options
+		// --------------------------------------------------------------------------------------------
+
+		/**
+		 * Prefix for properties delegated to the
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer} and
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer}.
+		 */
+		public static final String PROPERTIES_PREFIX = "properties.";
+
+		public static final ConfigOption<String> STREAM = ConfigOptions
+			.key("stream")
+			.stringType()
+			.noDefaultValue()
+			.withDescription("Name of the Kinesis stream backing this table (required)");
+
+		// --------------------------------------------------------------------------------------------
+		// Sink specific options
+		// --------------------------------------------------------------------------------------------
+
+		public static final ConfigOption<String> SINK_PARTITIONER = ConfigOptions
+			.key("sink.partitioner")
+			.stringType()
+			.noDefaultValue()
+			.withDescription(
+				"Optional output partitioning from Flink's partitions into Kinesis shards. "
+					+ "Sinks that write to tables defined with the PARTITION BY clause "
+					+ "always use a field-based partitioner and cannot define this option. "
+					+ "Valid enumerations are: \n"
+					+ "\"random\":"
+					+ " (use a random partition key),\n"
+					+ "\"fixed\":"
+					+ " (each Flink partition ends up in at most one Kinesis shard),\n"
+					+ "\"custom class name\":"
+					+ " (use a custom KinesisPartitioner subclass)");

Review comment:
       use a fully qualified name such that user can find this class

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisOptions.java
##########
@@ -0,0 +1,186 @@
+	/*
+	 * Licensed to the Apache Software Foundation (ASF) under one
+	 * or more contributor license agreements.  See the NOTICE file
+	 * distributed with this work for additional information
+	 * regarding copyright ownership.  The ASF licenses this file
+	 * to you under the Apache License, Version 2.0 (the
+	 * "License"); you may not use this file except in compliance
+	 * with the License.  You may obtain a copy of the License at
+	 *
+	 *     http://www.apache.org/licenses/LICENSE-2.0
+	 *
+	 * Unless required by applicable law or agreed to in writing, software
+	 * distributed under the License is distributed on an "AS IS" BASIS,
+	 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+	 * See the License for the specific language governing permissions and
+	 * limitations under the License.
+	 */
+
+	package org.apache.flink.streaming.connectors.kinesis.table;
+
+	import org.apache.flink.annotation.Internal;
+	import org.apache.flink.configuration.ConfigOption;
+	import org.apache.flink.configuration.ConfigOptions;
+	import org.apache.flink.configuration.ReadableConfig;
+	import org.apache.flink.streaming.connectors.kinesis.FixedKinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+	import org.apache.flink.table.api.ValidationException;
+	import org.apache.flink.table.catalog.CatalogTable;
+	import org.apache.flink.table.data.RowData;
+	import org.apache.flink.util.FlinkException;
+	import org.apache.flink.util.InstantiationUtil;
+
+	import java.util.Map;
+	import java.util.Properties;
+
+	/**
+	 * Options for Kinesis tables supported by the {@code CREATE TABLE ... WITH ...} clause of the
+	 * Flink SQL dialect and the Flink Table API.
+	 */
+	@Internal
+	public class KinesisOptions {
+
+		private KinesisOptions() {
+		}
+
+		// --------------------------------------------------------------------------------------------
+		// Kinesis specific options
+		// --------------------------------------------------------------------------------------------
+
+		/**
+		 * Prefix for properties delegated to the
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer} and
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer}.
+		 */
+		public static final String PROPERTIES_PREFIX = "properties.";
+
+		public static final ConfigOption<String> STREAM = ConfigOptions
+			.key("stream")
+			.stringType()
+			.noDefaultValue()
+			.withDescription("Name of the Kinesis stream backing this table (required)");
+
+		// --------------------------------------------------------------------------------------------
+		// Sink specific options
+		// --------------------------------------------------------------------------------------------
+
+		public static final ConfigOption<String> SINK_PARTITIONER = ConfigOptions
+			.key("sink.partitioner")
+			.stringType()
+			.noDefaultValue()
+			.withDescription(
+				"Optional output partitioning from Flink's partitions into Kinesis shards. "
+					+ "Sinks that write to tables defined with the PARTITION BY clause "
+					+ "always use a field-based partitioner and cannot define this option. "
+					+ "Valid enumerations are: \n"
+					+ "\"random\":"
+					+ " (use a random partition key),\n"
+					+ "\"fixed\":"
+					+ " (each Flink partition ends up in at most one Kinesis shard),\n"
+					+ "\"custom class name\":"
+					+ " (use a custom KinesisPartitioner subclass)");
+
+		public static final ConfigOption<String> SINK_PARTITIONER_FIELD_DELIMITER = ConfigOptions
+			.key("sink.partitioner.field.delimiter")

Review comment:
       call this `sink.partitioner-field-delimiter`? the reason for this is that we try to keep the options hierarchical (imagine a JSON representation). In this case `sink.partitioner` and `sink.partitioner.field` could not be represented.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * <p>A {@link KinesisPartitioner} of {@link RowData} elements that constructs the partition key
+ * from a list of field names.</p>
+ *
+ * <p>The key is constructed by concatenating the string representations of a list of fields
+ * projected from an input element. A fixed prefix can be optionally configured in order to speed
+ * up the key construction process.</p>
+ *
+ * <p>Resulting partition key values are trimmed to the maximum length allowed by Kinesis.</p>
+ */
+@Internal
+public class RowDataFieldsKinesisPartitioner extends KinesisPartitioner<RowData> {
+
+	/**
+	 * Allowed maximum length limit of a partition key.
+	 *
+	 * @link https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#API_PutRecord_RequestSyntax
+	 */
+	public static final int MAX_PARTITION_KEY_LENGTH = 256;
+
+	/**
+	 * Default delimiter for {@link RowDataFieldsKinesisPartitioner#delimiter}.
+	 */
+	public static final String DEFAULT_DELIMITER = String.valueOf('|');
+
+	/**
+	 * The character used to delimit field values in the concatenated partition key string.
+	 */
+	private final String delimiter;
+
+	/**
+	 * A list of field names used to extract the partition key for a record that will be written to
+	 * a Kinesis stream.
+	 */
+	private final List<String> fieldNames;
+
+	/**
+	 * A list of getter functions to dynamically extract the field values for all
+	 * {@link RowDataFieldsKinesisPartitioner#fieldNames} from an input record.
+	 */
+	private final RowData.FieldGetter[] dynamicFieldGetters;
+
+	/**
+	 * A buffer used to accumulate the concatenation of all field values that form the partition
+	 * key.
+	 */
+	private final StringBuilder keyBuffer = new StringBuilder();
+
+	/**
+	 * A prefix of fixed values to be used instead of the corresponding
+	 * {@link RowDataFieldsKinesisPartitioner#dynamicFieldGetters} entries.
+	 */
+	private String[] fixedPrefix = new String[]{};
+
+	/**
+	 * The cumulative length of all values currently stored in the
+	 * {@link RowDataFieldsKinesisPartitioner#fixedPrefix}.
+	 */
+	private int fixedPrefixLength = 0;
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table) {
+		this(table, DEFAULT_DELIMITER);
+	}
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table, String delimiter) {
+		Preconditions.checkNotNull(table, "table");
+		Preconditions.checkNotNull(delimiter, "delimiter");
+		Preconditions.checkArgument(
+			table.isPartitioned(),
+			"Cannot create a RowDataFieldsKinesisPartitioner for a non-partitioned table");
+		Preconditions.checkArgument(
+			table.getPartitionKeys().size() == new HashSet<>(table.getPartitionKeys()).size(),
+			"The sequence of partition keys cannot contain duplicates");
+
+		TableSchema schema = table.getSchema();
+		List<String> schemaFieldsList = Arrays.asList(schema.getFieldNames());
+
+		String[] badKeys = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				boolean hasField = schema.getTableColumn(fieldName).isPresent();
+				return hasField ? Stream.empty() : Stream.of(fieldName);
+			})
+			.toArray(String[]::new);
+		Preconditions.checkArgument(
+			badKeys.length == 0,
+			"The following partition keys are not present in the table: %s",
+			String.join(", ", badKeys));
+
+		this.delimiter = delimiter;
+		this.fieldNames = table.getPartitionKeys();
+		this.dynamicFieldGetters = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				Optional<TableColumn> tableColumn = schema.getTableColumn(fieldName);
+				// convert Optional to Stream to unpack the value in the enclosing flatMap
+				return tableColumn.map(Stream::of).orElseGet(Stream::empty);
+			})
+			.map(column -> {
+				LogicalType fieldType = column.getType().getLogicalType();
+				int fieldPos = schemaFieldsList.indexOf(column.getName());
+				return RowData.createFieldGetter(fieldType, fieldPos);
+			})
+			.toArray(RowData.FieldGetter[]::new);
+	}
+
+	@Override
+	public String getPartitionId(RowData element) {
+		keyBuffer.setLength(fixedPrefixLength); // reset the dynamic part of the key buffer
+		for (int i = fixedPrefix.length; i < dynamicFieldGetters.length; i++) {
+			RowData.FieldGetter getter = dynamicFieldGetters[i];
+			Object fieldValue = getter.getFieldOrNull(element);
+
+			keyBuffer.append(fieldValue);

Review comment:
       are we limiting the supported data types for partitions somewhere? we cannot assume that every data type (esp in internal data structures) has a good toString method. If I check `SupportsPartitioning`, it seems we always limited this STRING type?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicSource.java
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+
+/**
+ * Kinesis-backed {@link ScanTableSource}.
+ */
+@Internal
+public class KinesisDynamicSource implements ScanTableSource, SupportsReadingMetadata {
+
+	/** List of read-only metadata fields that the source can provide upstream upon request. */
+	private static final Map<String, DataType> READABLE_METADATA =
+		new HashMap<String, DataType>() {{
+			for (Metadata metadata : Metadata.values()) {
+				put(metadata.name(), metadata.getDataType());
+			}
+		}};
+
+	// --------------------------------------------------------------------------------------------
+	// Mutable attributes
+	// --------------------------------------------------------------------------------------------
+
+	/** Data type that describes the final output of the source. */
+	private DataType producedDataType;
+
+	/** Metadata that is requested to be appended at the end of a physical source row. */
+	private List<Metadata> requestedMetadataFields;
+
+	// --------------------------------------------------------------------------------------------
+	// Scan format attributes
+	// --------------------------------------------------------------------------------------------
+
+	/** Data type to configure the format. */
+	private final DataType physicalDataType;
+
+	/** Scan format for decoding records from Kinesis. */
+	private final DecodingFormat<DeserializationSchema<RowData>> decodingFormat;
+
+	// --------------------------------------------------------------------------------------------
+	// Kinesis-specific attributes
+	// --------------------------------------------------------------------------------------------
+
+	/** The Kinesis stream to consume. */
+	private final String stream;
+
+	/** Properties for the Kinesis consumer. */
+	private final Properties consumerProperties;
+
+	public KinesisDynamicSource(
+		DataType physicalDataType,
+		String stream,
+		Properties consumerProperties,
+		DecodingFormat<DeserializationSchema<RowData>> decodingFormat) {
+		this(
+			physicalDataType,
+			stream,
+			consumerProperties,
+			decodingFormat,
+			null,
+			Collections.emptyList());
+	}
+
+	public KinesisDynamicSource(
+		DataType physicalDataType,
+		String stream,
+		Properties consumerProperties,
+		DecodingFormat<DeserializationSchema<RowData>> decodingFormat,
+		@Nullable DataType producedDataType,
+		List<Metadata> requestedMetadataFields) {
+
+		this.producedDataType = this.physicalDataType = Preconditions.checkNotNull(
+			physicalDataType,
+			"Produced data type must not be null.");
+		this.stream = Preconditions.checkNotNull(
+			stream,
+			"Stream must not be null.");
+		this.consumerProperties = Preconditions.checkNotNull(
+			consumerProperties,
+			"Properties must not be null.");
+		this.decodingFormat = Preconditions.checkNotNull(
+			decodingFormat,
+			"Decoding format must not be null.");
+		this.producedDataType = producedDataType;

Review comment:
       remove this

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FixedKinesisPartitioner.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Objects;
+
+/**
+ * A partitioner ensuring that each internal Flink partition ends up in the same Kinesis partition.
+ *
+ * <p>This is achieved by using the index of the producer task as a {@code PartitionKey}.</p>
+ */
+@PublicEvolving
+public class FixedKinesisPartitioner<T> extends KinesisPartitioner<T> {

Review comment:
       nit: make partitioner classes final?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactory.java
##########
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Factory for creating {@link KinesisDynamicSource} and {@link KinesisDynamicSink} instances.
+ */
+@Internal
+public class KinesisDynamicTableFactory implements
+	DynamicTableSourceFactory,
+	DynamicTableSinkFactory {
+	public static final String IDENTIFIER = "kinesis";
+
+	// --------------------------------------------------------------------------------------------
+	// DynamicTableSourceFactory
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public DynamicTableSource createDynamicTableSource(Context context) {
+		FactoryUtil.TableFactoryHelper helper = FactoryUtil
+			.createTableFactoryHelper(this, context);
+		Properties connectorProperties = KinesisOptions
+			.getConnectorProperties(context.getCatalogTable().getOptions());
+		ReadableConfig tableOptions = helper.getOptions();
+
+		// initialize the table format early in order to register its consumedOptionKeys
+		// in the TableFactoryHelper, as those are needed for correct option validation
+		DecodingFormat<DeserializationSchema<RowData>> decodingFormat = helper
+			.discoverDecodingFormat(DeserializationFormatFactory.class, FactoryUtil.FORMAT);
+
+		// Validate option data types
+		helper.validateExcept(KinesisOptions.PROPERTIES_PREFIX);
+		// Validate option values
+		validateTableOptionsAndConnectorProperties(tableOptions, connectorProperties);
+
+		return new KinesisDynamicSource(
+			context.getCatalogTable().getSchema().toPhysicalRowDataType(),
+			tableOptions.get(KinesisOptions.STREAM),
+			KinesisOptions.getConnectorProperties(context.getCatalogTable().getOptions()),

Review comment:
       reuse variable

##########
File path: docs/dev/table/connectors/kinesis.md
##########
@@ -0,0 +1,334 @@
+---
+title: "Amazon Kinesis Data Streams SQL Connector"
+nav-title: Kinesis
+nav-parent_id: sql-connectors
+nav-pos: 2
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<span class="label label-primary">Scan Source: Unbounded</span>
+<span class="label label-primary">Sink: Streaming Append Mode</span>
+
+* This will be replaced by the TOC
+{:toc}
+
+The Kinesis connector allows for reading data from and writing data into [Amazon Kinesis Data Streams (KDS)](https://aws.amazon.com/kinesis/data-streams/).
+
+Dependencies
+------------
+
+To use the connector, add the following Maven dependency to your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-connector-kinesis{{ site.scala_version_suffix }}</artifactId>
+  <version>{{site.version }}</version>
+</dependency>
+{% endhighlight %}
+
+How to create a Kinesis data stream table
+-----------------------------------------
+
+Follow the instructions from the [Amazon KDS Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html) to set up a Kinesis stream.
+The following example shows how to create a table backed by a Kinesis data stream:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3)
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Available Metadata
+------------------
+
+The following metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Column Name</th>
+      <th class="text-center" style="width: 45%">Column Type</th>
+      <th class="text-center" style="width: 35%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-ApproximateArrivalTimestamp">ApproximateArrivalTimestamp</a></code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIMEZONE NOT NULL</code></td>
+      <td>The approximate time that the record was inserted into the stream.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Shard.html#Streams-Type-Shard-ShardId">ShardId</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the shard within the stream from which the record was read.</td>
+    </tr>
+    <tr>
+      <td><code><a href="https://docs.aws.amazon.com/kinesis/latest/APIReference/API_Record.html#Streams-Type-Record-SequenceNumber">SequenceNumber</a></code></td>
+      <td><code>VARCHAR(128) NOT NULL</code></td>
+      <td>The unique identifier of the record within its shard.</td>
+    </tr>
+    </tbody>
+</table>
+
+The extended `CREATE TABLE` example demonstrates the syntax for exposing these metadata columns:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KinesisTable (
+ user_id BIGINT,
+ item_id BIGINT,
+ category_id BIGINT,
+ behavior STRING,
+ ts TIMESTAMP(3),
+ arrival_time TIMESTAMP(3) METADATA FROM 'ApproximateArrivalTimestamp' VIRTUAL,
+ shard_id VARCHAR(128) NOT NULL METADATA FROM 'ShardId' VIRTUAL,
+ sequence_number VARCHAR(128) NOT NULL METADATA FROM 'SequenceNumber' VIRTUAL
+) PARTITIONED BY (user_id, item_id) WITH (
+ 'connector' = 'kinesis',
+ 'stream' = 'user_behavior',
+ 'properties.aws.region' = 'us-east-2',
+ 'properties.flink.stream.initpos' = 'LATEST',
+ 'format' = 'csv'
+)
+{% endhighlight %}
+</div>
+</div>
+
+Connector Options
+-----------------
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Option</th>
+      <th class="text-center" style="width: 8%">Required</th>
+      <th class="text-center" style="width: 7%">Default</th>
+      <th class="text-center" style="width: 10%">Type</th>
+      <th class="text-center" style="width: 50%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><h5>connector</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Specify what connector to use. For Kinesis use <code>'kinesis'</code>.</td>
+    </tr>
+    <tr>
+      <td><h5>stream</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>Name of the Kinesis data stream backing this table.</td>
+    </tr>
+    <tr>
+      <td><h5>format</h5></td>
+      <td>required</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The format used to deserialize and serialize Kinesis data stream records. See <a href="#data-type-mapping">Data Type Mapping</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">random or row-based</td>
+      <td>String</td>
+      <td>Optional output partitioning from Flink's partitions into Kinesis shards. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>sink.partitioner.field.delimiter</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">|</td>
+      <td>String</td>
+      <td>Optional field delimiter for a fields-based partitioner derived from a PARTITION BY clause. See <a href="#sink-partitioning">Sink Partitioning</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.region</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS region where the stream is defined. Either this or <code>properties.aws.endpoint</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.endpoint</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The AWS endpoint for Kinesis (derived from the AWS region setting if not set). Either this or <code>properties.aws.region</code> are required.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.aws.credentials.provider</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">AUTO</td>
+      <td>String</td>
+      <td>A credentials provider to use when authenticating against the Kinesis endpoint. See <a href="#authentication">Authentication</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.initpos</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">LATEST</td>
+      <td>String</td>
+      <td>Initial position to be used when reading from the table. See <a href="#start-reading-position">Start Reading Position</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.flink.stream.recordpublisher</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">POLLING</td>
+      <td>String</td>
+      <td>The `RecordPublisher` type to use for sources. See <a href="#enhanced-fan-out">Enhanced Fan-Out</a> for details.</td>
+    </tr>
+    <tr>
+      <td><h5>properties.*</h5></td>
+      <td>optional</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>
+      Other properties to pass to the FlinkKinesisConsumer or FlinkKinesisProducer constructors. 
+      See the constants defined in 
+      <ul>
+        <li><code>AWSConfigConstants</code>,</li> 
+        <li><code>ConsumerConfigConstants</code>, and</li> 
+        <li><code>ProducerConfigConstants</code></li>
+      </ul>
+      for detailed information on the available suffixes.
+      </td>
+    </tr>
+    </tbody>
+</table>
+
+Features
+--------
+
+### Authorization
+
+Make sure to [create an appropriate IAM policy](https://docs.aws.amazon.com/streams/latest/dev/controlling-access.html) to allow reading from / writing to the Kinesis data streams.
+
+### Authentication
+
+Depending on your deployment you would choose a different Credentials Provider to allow access to Kinesis.
+By default, the `AUTO` Credentials Provider is used.
+If the access key ID and secret key are set in the deployment configuration, this results in using the `BASIC` provider.  
+
+A specific [AWSCredentialsProvider](https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/index.html?com/amazonaws/auth/AWSCredentialsProvider.html) can be **optionally** set using the `properties.aws.credentials.provider` setting.
+Supported values are:
+
+* `AUTO` - Use the default AWS Credentials Provider chain that searches for credentials in the following order: `ENV_VARS`, `SYS_PROPS`, `WEB_IDENTITY_TOKEN`, `PROFILE`, and EC2/ECS credentials provider.
+* `BASIC` - Use access key ID and secret key supplied as configuration. 
+* `ENV_VAR` - Use `AWS_ACCESS_KEY_ID` & `AWS_SECRET_ACCESS_KEY` environment variables.
+* `SYS_PROP` - Use Java system properties `aws.accessKeyId` and `aws.secretKey`.
+* `PROFILE` - Use an AWS credentials profile to create the AWS credentials.
+* `ASSUME_ROLE` - Create AWS credentials by assuming a role. The credentials for assuming the role must be supplied.
+* `WEB_IDENTITY_TOKEN` - Create AWS credentials by assuming a role using Web Identity Token. 
+
+### Start Reading Position
+
+You can configure table sources to start reading a table-backing Kinesis data stream from a specific position through the `properties.flink.stream.initpos` option.

Review comment:
       I'm wondering if we should also improve those properties for the table connector. `properties.flink.stream.initpos` is very long and has even `flink` in its name. Ideally, we split the properties into `scan.` and `sink.`. See https://cwiki.apache.org/confluence/display/FLINK/FLIP-122%3A+New+Connector+Property+Keys+for+New+Factory for more information.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisOptions.java
##########
@@ -0,0 +1,186 @@
+	/*

Review comment:
       this class uses wrong indention 

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisOptions.java
##########
@@ -0,0 +1,186 @@
+	/*
+	 * Licensed to the Apache Software Foundation (ASF) under one
+	 * or more contributor license agreements.  See the NOTICE file
+	 * distributed with this work for additional information
+	 * regarding copyright ownership.  The ASF licenses this file
+	 * to you under the Apache License, Version 2.0 (the
+	 * "License"); you may not use this file except in compliance
+	 * with the License.  You may obtain a copy of the License at
+	 *
+	 *     http://www.apache.org/licenses/LICENSE-2.0
+	 *
+	 * Unless required by applicable law or agreed to in writing, software
+	 * distributed under the License is distributed on an "AS IS" BASIS,
+	 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+	 * See the License for the specific language governing permissions and
+	 * limitations under the License.
+	 */
+
+	package org.apache.flink.streaming.connectors.kinesis.table;
+
+	import org.apache.flink.annotation.Internal;
+	import org.apache.flink.configuration.ConfigOption;
+	import org.apache.flink.configuration.ConfigOptions;
+	import org.apache.flink.configuration.ReadableConfig;
+	import org.apache.flink.streaming.connectors.kinesis.FixedKinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+	import org.apache.flink.table.api.ValidationException;
+	import org.apache.flink.table.catalog.CatalogTable;
+	import org.apache.flink.table.data.RowData;
+	import org.apache.flink.util.FlinkException;
+	import org.apache.flink.util.InstantiationUtil;
+
+	import java.util.Map;
+	import java.util.Properties;
+
+	/**
+	 * Options for Kinesis tables supported by the {@code CREATE TABLE ... WITH ...} clause of the
+	 * Flink SQL dialect and the Flink Table API.
+	 */
+	@Internal
+	public class KinesisOptions {

Review comment:
       make it default scoped? it is internal anyways

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisOptions.java
##########
@@ -0,0 +1,186 @@
+	/*
+	 * Licensed to the Apache Software Foundation (ASF) under one
+	 * or more contributor license agreements.  See the NOTICE file
+	 * distributed with this work for additional information
+	 * regarding copyright ownership.  The ASF licenses this file
+	 * to you under the Apache License, Version 2.0 (the
+	 * "License"); you may not use this file except in compliance
+	 * with the License.  You may obtain a copy of the License at
+	 *
+	 *     http://www.apache.org/licenses/LICENSE-2.0
+	 *
+	 * Unless required by applicable law or agreed to in writing, software
+	 * distributed under the License is distributed on an "AS IS" BASIS,
+	 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+	 * See the License for the specific language governing permissions and
+	 * limitations under the License.
+	 */
+
+	package org.apache.flink.streaming.connectors.kinesis.table;
+
+	import org.apache.flink.annotation.Internal;
+	import org.apache.flink.configuration.ConfigOption;
+	import org.apache.flink.configuration.ConfigOptions;
+	import org.apache.flink.configuration.ReadableConfig;
+	import org.apache.flink.streaming.connectors.kinesis.FixedKinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+	import org.apache.flink.table.api.ValidationException;
+	import org.apache.flink.table.catalog.CatalogTable;
+	import org.apache.flink.table.data.RowData;
+	import org.apache.flink.util.FlinkException;
+	import org.apache.flink.util.InstantiationUtil;
+
+	import java.util.Map;
+	import java.util.Properties;
+
+	/**
+	 * Options for Kinesis tables supported by the {@code CREATE TABLE ... WITH ...} clause of the
+	 * Flink SQL dialect and the Flink Table API.
+	 */
+	@Internal
+	public class KinesisOptions {
+
+		private KinesisOptions() {
+		}
+
+		// --------------------------------------------------------------------------------------------
+		// Kinesis specific options
+		// --------------------------------------------------------------------------------------------
+
+		/**
+		 * Prefix for properties delegated to the
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer} and
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer}.
+		 */
+		public static final String PROPERTIES_PREFIX = "properties.";
+
+		public static final ConfigOption<String> STREAM = ConfigOptions
+			.key("stream")
+			.stringType()
+			.noDefaultValue()
+			.withDescription("Name of the Kinesis stream backing this table (required)");
+
+		// --------------------------------------------------------------------------------------------
+		// Sink specific options
+		// --------------------------------------------------------------------------------------------
+
+		public static final ConfigOption<String> SINK_PARTITIONER = ConfigOptions
+			.key("sink.partitioner")
+			.stringType()
+			.noDefaultValue()
+			.withDescription(
+				"Optional output partitioning from Flink's partitions into Kinesis shards. "
+					+ "Sinks that write to tables defined with the PARTITION BY clause "
+					+ "always use a field-based partitioner and cannot define this option. "
+					+ "Valid enumerations are: \n"
+					+ "\"random\":"
+					+ " (use a random partition key),\n"
+					+ "\"fixed\":"
+					+ " (each Flink partition ends up in at most one Kinesis shard),\n"
+					+ "\"custom class name\":"
+					+ " (use a custom KinesisPartitioner subclass)");
+
+		public static final ConfigOption<String> SINK_PARTITIONER_FIELD_DELIMITER = ConfigOptions
+			.key("sink.partitioner.field.delimiter")
+			.stringType()
+			.defaultValue("|")
+			.withDescription(
+				"Optional field delimiter for fields-based partitioner "
+					+ "derived from a PARTITION BY clause (\"|\" by default)");
+
+		// --------------------------------------------------------------------------------------------
+		// Option enumerations
+		// --------------------------------------------------------------------------------------------
+
+		public static final String SINK_PARTITIONER_VALUE_FIXED = "fixed";
+
+		public static final String SINK_PARTITIONER_VALUE_RANDOM = "random";
+
+		// --------------------------------------------------------------------------------------------
+		// Utilities
+		// --------------------------------------------------------------------------------------------
+
+		public static Properties getConnectorProperties(Map<String, String> tableOptions) {
+			final Properties properties = new Properties();
+
+			tableOptions.keySet().stream()
+				.filter(key -> key.startsWith(PROPERTIES_PREFIX))
+				.forEach(key -> {
+					final String value = tableOptions.get(key);
+					final String subKey = key.substring((PROPERTIES_PREFIX).length());
+					properties.put(subKey, value);
+				});
+
+			return properties;
+		}
+
+		/**
+		 * Constructs the kinesis partitioner for a {@code targetTable} based on the currently set
+		 * {@code tableOptions}.
+		 *
+		 * <p>The following rules are applied with decreasing precedence order.</p>
+		 *
+		 * <ul>
+		 *   <li>IF {@code targetTable} is partitioned, return a {@code RowDataKinesisPartitioner}.</li>
+		 *   <li>If the partitioner type is not set, return a {@link RandomKinesisPartitioner}.</li>
+		 *   <li>If a specific partitioner type alias is used, instantiate the corresponding type</li>
+		 *   <li>Interpret the partitioner type as a classname of a user-defiend partitioner.</li>
+		 * </ul>
+		 *
+		 * @param tableOptions A read-only set of config options that determines the partitioner type.
+		 * @param targetTable A catalog version of the table backing the partitioner.
+		 * @param classLoader A {@link ClassLoader} to use for loading user-defined partitioner classes.
+		 */
+		public static KinesisPartitioner<RowData> getKinesisPartitioner(
+			ReadableConfig tableOptions,
+			CatalogTable targetTable,
+			ClassLoader classLoader) {
+
+			if (targetTable.isPartitioned()) {
+				String delimiter = tableOptions.get(SINK_PARTITIONER_FIELD_DELIMITER);
+				return new RowDataFieldsKinesisPartitioner(targetTable, delimiter);
+			} else if (!tableOptions.getOptional(SINK_PARTITIONER).isPresent()) {
+				return new RandomKinesisPartitioner<>();
+			} else {
+				String partitioner = tableOptions.getOptional(SINK_PARTITIONER).get();
+				if (SINK_PARTITIONER_VALUE_FIXED.equals(partitioner)) {
+					return new FixedKinesisPartitioner<>();
+				} else if (SINK_PARTITIONER_VALUE_RANDOM.equals(partitioner)) {
+					return new RandomKinesisPartitioner<>();
+				} else { // interpret the option value as a fully-qualified class name
+					return initializePartitioner(partitioner, classLoader);
+				}
+			}
+
+		}
+
+		/**
+		 * Returns a class value with the given class name.
+		 */
+		private static <T> KinesisPartitioner<T> initializePartitioner(
+			String name,
+			ClassLoader classLoader) {
+			try {
+				Class<?> clazz = Class.forName(name, true, classLoader);
+				if (!KinesisPartitioner.class.isAssignableFrom(clazz)) {

Review comment:
       use `instantiate(Class<T> clazz, Class<? super T> castTo)`

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * <p>A {@link KinesisPartitioner} of {@link RowData} elements that constructs the partition key
+ * from a list of field names.</p>

Review comment:
       nit: `</p>` are not necessary in JavaDocs

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisOptions.java
##########
@@ -0,0 +1,186 @@
+	/*
+	 * Licensed to the Apache Software Foundation (ASF) under one
+	 * or more contributor license agreements.  See the NOTICE file
+	 * distributed with this work for additional information
+	 * regarding copyright ownership.  The ASF licenses this file
+	 * to you under the Apache License, Version 2.0 (the
+	 * "License"); you may not use this file except in compliance
+	 * with the License.  You may obtain a copy of the License at
+	 *
+	 *     http://www.apache.org/licenses/LICENSE-2.0
+	 *
+	 * Unless required by applicable law or agreed to in writing, software
+	 * distributed under the License is distributed on an "AS IS" BASIS,
+	 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+	 * See the License for the specific language governing permissions and
+	 * limitations under the License.
+	 */
+
+	package org.apache.flink.streaming.connectors.kinesis.table;
+
+	import org.apache.flink.annotation.Internal;
+	import org.apache.flink.configuration.ConfigOption;
+	import org.apache.flink.configuration.ConfigOptions;
+	import org.apache.flink.configuration.ReadableConfig;
+	import org.apache.flink.streaming.connectors.kinesis.FixedKinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+	import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+	import org.apache.flink.table.api.ValidationException;
+	import org.apache.flink.table.catalog.CatalogTable;
+	import org.apache.flink.table.data.RowData;
+	import org.apache.flink.util.FlinkException;
+	import org.apache.flink.util.InstantiationUtil;
+
+	import java.util.Map;
+	import java.util.Properties;
+
+	/**
+	 * Options for Kinesis tables supported by the {@code CREATE TABLE ... WITH ...} clause of the
+	 * Flink SQL dialect and the Flink Table API.
+	 */
+	@Internal
+	public class KinesisOptions {
+
+		private KinesisOptions() {
+		}
+
+		// --------------------------------------------------------------------------------------------
+		// Kinesis specific options
+		// --------------------------------------------------------------------------------------------
+
+		/**
+		 * Prefix for properties delegated to the
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer} and
+		 * {@link org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer}.
+		 */
+		public static final String PROPERTIES_PREFIX = "properties.";
+
+		public static final ConfigOption<String> STREAM = ConfigOptions
+			.key("stream")
+			.stringType()
+			.noDefaultValue()
+			.withDescription("Name of the Kinesis stream backing this table (required)");
+
+		// --------------------------------------------------------------------------------------------
+		// Sink specific options
+		// --------------------------------------------------------------------------------------------
+
+		public static final ConfigOption<String> SINK_PARTITIONER = ConfigOptions
+			.key("sink.partitioner")
+			.stringType()
+			.noDefaultValue()
+			.withDescription(
+				"Optional output partitioning from Flink's partitions into Kinesis shards. "
+					+ "Sinks that write to tables defined with the PARTITION BY clause "
+					+ "always use a field-based partitioner and cannot define this option. "
+					+ "Valid enumerations are: \n"
+					+ "\"random\":"
+					+ " (use a random partition key),\n"
+					+ "\"fixed\":"
+					+ " (each Flink partition ends up in at most one Kinesis shard),\n"
+					+ "\"custom class name\":"
+					+ " (use a custom KinesisPartitioner subclass)");
+
+		public static final ConfigOption<String> SINK_PARTITIONER_FIELD_DELIMITER = ConfigOptions
+			.key("sink.partitioner.field.delimiter")
+			.stringType()
+			.defaultValue("|")
+			.withDescription(
+				"Optional field delimiter for fields-based partitioner "
+					+ "derived from a PARTITION BY clause (\"|\" by default)");
+
+		// --------------------------------------------------------------------------------------------
+		// Option enumerations
+		// --------------------------------------------------------------------------------------------
+
+		public static final String SINK_PARTITIONER_VALUE_FIXED = "fixed";
+
+		public static final String SINK_PARTITIONER_VALUE_RANDOM = "random";
+
+		// --------------------------------------------------------------------------------------------
+		// Utilities
+		// --------------------------------------------------------------------------------------------
+
+		public static Properties getConnectorProperties(Map<String, String> tableOptions) {
+			final Properties properties = new Properties();
+
+			tableOptions.keySet().stream()
+				.filter(key -> key.startsWith(PROPERTIES_PREFIX))
+				.forEach(key -> {
+					final String value = tableOptions.get(key);
+					final String subKey = key.substring((PROPERTIES_PREFIX).length());
+					properties.put(subKey, value);
+				});
+
+			return properties;
+		}
+
+		/**
+		 * Constructs the kinesis partitioner for a {@code targetTable} based on the currently set
+		 * {@code tableOptions}.
+		 *
+		 * <p>The following rules are applied with decreasing precedence order.</p>
+		 *
+		 * <ul>
+		 *   <li>IF {@code targetTable} is partitioned, return a {@code RowDataKinesisPartitioner}.</li>

Review comment:
       nit: some typos `IF`, `user-defiend`

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * <p>A {@link KinesisPartitioner} of {@link RowData} elements that constructs the partition key
+ * from a list of field names.</p>
+ *
+ * <p>The key is constructed by concatenating the string representations of a list of fields
+ * projected from an input element. A fixed prefix can be optionally configured in order to speed
+ * up the key construction process.</p>
+ *
+ * <p>Resulting partition key values are trimmed to the maximum length allowed by Kinesis.</p>
+ */
+@Internal
+public class RowDataFieldsKinesisPartitioner extends KinesisPartitioner<RowData> {
+

Review comment:
       add a `serialVersionUID` to all partitioners

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataKinesisDeserializationSchema.java
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A {@link KinesisDeserializationSchema} adaptor for {@link RowData} records that delegates
+ * physical data deserialization to an inner {@link DeserializationSchema} and appends requested
+ * metadata to the end of the deserialized {@link RowData} record.
+ */
+@Internal
+public class RowDataKinesisDeserializationSchema implements KinesisDeserializationSchema<RowData> {
+
+	/** Internal type for enumerating available metadata. */
+	protected enum Metadata {
+		ApproximateArrivalTimestamp(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull()),
+		SequenceNumber(DataTypes.VARCHAR(128).notNull()),
+		ShardId(DataTypes.VARCHAR(128).notNull());
+
+		private final DataType dataType;
+
+		Metadata(DataType dataType) {
+			this.dataType = dataType;
+		}
+
+		public DataType getDataType() {
+			return this.dataType;
+		}
+	}
+
+	private static final long serialVersionUID = 5551095193778230749L;
+
+	/** A {@link DeserializationSchema} to deserialize the physical part of the row. */
+	private final DeserializationSchema<RowData> physicalDeserializer;
+
+	/** The type of the produced {@link RowData} records (physical data with appended metadata]. */
+	private final TypeInformation<RowData> producedTypeInfo;
+
+	/** Metadata fields to be appended to the physical {@link RowData} in the produced records. */
+	private final List<Metadata> requestedMetadataFields;
+
+	public RowDataKinesisDeserializationSchema(
+		DeserializationSchema<RowData> physicalDeserializer,
+		TypeInformation<RowData> producedTypeInfo,
+		List<Metadata> requestedMetadataFields) {
+		this.physicalDeserializer = Preconditions.checkNotNull(physicalDeserializer);
+		this.producedTypeInfo = Preconditions.checkNotNull(producedTypeInfo);
+		this.requestedMetadataFields = Preconditions.checkNotNull(requestedMetadataFields);
+	}
+
+	@Override
+	public void open(DeserializationSchema.InitializationContext context) throws Exception {
+		physicalDeserializer.open(context);
+	}
+
+	@Override
+	public RowData deserialize(
+		byte[] recordValue,
+		String partitionKey,
+		String seqNum,
+		long approxArrivalTimestamp,
+		String stream,
+		String shardId) throws IOException {
+
+		RowData physicalRow = physicalDeserializer.deserialize(recordValue);

Review comment:
       We should call `deserialize(byte[] message, Collector<T> out)` instead. 

##########
File path: flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestFormatFactory.java
##########
@@ -140,8 +142,28 @@ public int hashCode() {
 		}
 	}
 
+	/**
+	 * {@link DeserializationSchema} for testing.
+	 */
+	public static class DeserializationSchemaMock implements DeserializationSchema<RowData> {

Review comment:
       keep it private for now?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * <p>A {@link KinesisPartitioner} of {@link RowData} elements that constructs the partition key
+ * from a list of field names.</p>
+ *
+ * <p>The key is constructed by concatenating the string representations of a list of fields
+ * projected from an input element. A fixed prefix can be optionally configured in order to speed
+ * up the key construction process.</p>
+ *
+ * <p>Resulting partition key values are trimmed to the maximum length allowed by Kinesis.</p>
+ */
+@Internal
+public class RowDataFieldsKinesisPartitioner extends KinesisPartitioner<RowData> {
+
+	/**
+	 * Allowed maximum length limit of a partition key.
+	 *
+	 * @link https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#API_PutRecord_RequestSyntax
+	 */
+	public static final int MAX_PARTITION_KEY_LENGTH = 256;
+
+	/**
+	 * Default delimiter for {@link RowDataFieldsKinesisPartitioner#delimiter}.
+	 */
+	public static final String DEFAULT_DELIMITER = String.valueOf('|');
+
+	/**
+	 * The character used to delimit field values in the concatenated partition key string.
+	 */
+	private final String delimiter;
+
+	/**
+	 * A list of field names used to extract the partition key for a record that will be written to
+	 * a Kinesis stream.
+	 */
+	private final List<String> fieldNames;
+
+	/**
+	 * A list of getter functions to dynamically extract the field values for all
+	 * {@link RowDataFieldsKinesisPartitioner#fieldNames} from an input record.
+	 */
+	private final RowData.FieldGetter[] dynamicFieldGetters;
+
+	/**
+	 * A buffer used to accumulate the concatenation of all field values that form the partition
+	 * key.
+	 */
+	private final StringBuilder keyBuffer = new StringBuilder();
+
+	/**
+	 * A prefix of fixed values to be used instead of the corresponding
+	 * {@link RowDataFieldsKinesisPartitioner#dynamicFieldGetters} entries.
+	 */
+	private String[] fixedPrefix = new String[]{};
+
+	/**
+	 * The cumulative length of all values currently stored in the
+	 * {@link RowDataFieldsKinesisPartitioner#fixedPrefix}.
+	 */
+	private int fixedPrefixLength = 0;
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table) {
+		this(table, DEFAULT_DELIMITER);
+	}
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table, String delimiter) {
+		Preconditions.checkNotNull(table, "table");
+		Preconditions.checkNotNull(delimiter, "delimiter");
+		Preconditions.checkArgument(
+			table.isPartitioned(),
+			"Cannot create a RowDataFieldsKinesisPartitioner for a non-partitioned table");
+		Preconditions.checkArgument(
+			table.getPartitionKeys().size() == new HashSet<>(table.getPartitionKeys()).size(),
+			"The sequence of partition keys cannot contain duplicates");
+
+		TableSchema schema = table.getSchema();
+		List<String> schemaFieldsList = Arrays.asList(schema.getFieldNames());
+
+		String[] badKeys = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				boolean hasField = schema.getTableColumn(fieldName).isPresent();

Review comment:
       This logic should not work on `TableSchema` but rather on `TableSchema.toPersistedDataType` because computed columns won't be present. Persisted metadata columns could be used in the partitioner.

##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);
+		actualSource.applyReadableMetadata(metadataKeys, producedDataType);
+
+		// verify that the constructed DynamicTableSink is as expected

Review comment:
       copy past error `DynamicTableSink` here and below

##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       we are not really testing something if both call the same methods with the same parameters, instead we should test the contents of mutable member variables

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * <p>A {@link KinesisPartitioner} of {@link RowData} elements that constructs the partition key
+ * from a list of field names.</p>
+ *
+ * <p>The key is constructed by concatenating the string representations of a list of fields
+ * projected from an input element. A fixed prefix can be optionally configured in order to speed
+ * up the key construction process.</p>
+ *
+ * <p>Resulting partition key values are trimmed to the maximum length allowed by Kinesis.</p>
+ */
+@Internal
+public class RowDataFieldsKinesisPartitioner extends KinesisPartitioner<RowData> {
+
+	/**
+	 * Allowed maximum length limit of a partition key.
+	 *
+	 * @link https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#API_PutRecord_RequestSyntax
+	 */
+	public static final int MAX_PARTITION_KEY_LENGTH = 256;
+
+	/**
+	 * Default delimiter for {@link RowDataFieldsKinesisPartitioner#delimiter}.
+	 */
+	public static final String DEFAULT_DELIMITER = String.valueOf('|');
+
+	/**
+	 * The character used to delimit field values in the concatenated partition key string.
+	 */
+	private final String delimiter;
+
+	/**
+	 * A list of field names used to extract the partition key for a record that will be written to
+	 * a Kinesis stream.
+	 */
+	private final List<String> fieldNames;
+
+	/**
+	 * A list of getter functions to dynamically extract the field values for all
+	 * {@link RowDataFieldsKinesisPartitioner#fieldNames} from an input record.
+	 */
+	private final RowData.FieldGetter[] dynamicFieldGetters;
+
+	/**
+	 * A buffer used to accumulate the concatenation of all field values that form the partition
+	 * key.
+	 */
+	private final StringBuilder keyBuffer = new StringBuilder();
+
+	/**
+	 * A prefix of fixed values to be used instead of the corresponding
+	 * {@link RowDataFieldsKinesisPartitioner#dynamicFieldGetters} entries.
+	 */
+	private String[] fixedPrefix = new String[]{};
+
+	/**
+	 * The cumulative length of all values currently stored in the
+	 * {@link RowDataFieldsKinesisPartitioner#fixedPrefix}.
+	 */
+	private int fixedPrefixLength = 0;
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table) {
+		this(table, DEFAULT_DELIMITER);
+	}
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table, String delimiter) {

Review comment:
       I wouldn't pass the entire `CatalogTable` to the partitioner. Instead the parameters should represent what is really needed. The constructor contains too much logic that should rather be put into the `KinesisOptions` utility methods.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * <p>A {@link KinesisPartitioner} of {@link RowData} elements that constructs the partition key
+ * from a list of field names.</p>
+ *
+ * <p>The key is constructed by concatenating the string representations of a list of fields
+ * projected from an input element. A fixed prefix can be optionally configured in order to speed
+ * up the key construction process.</p>
+ *
+ * <p>Resulting partition key values are trimmed to the maximum length allowed by Kinesis.</p>
+ */
+@Internal
+public class RowDataFieldsKinesisPartitioner extends KinesisPartitioner<RowData> {
+
+	/**
+	 * Allowed maximum length limit of a partition key.
+	 *
+	 * @link https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#API_PutRecord_RequestSyntax
+	 */
+	public static final int MAX_PARTITION_KEY_LENGTH = 256;
+
+	/**
+	 * Default delimiter for {@link RowDataFieldsKinesisPartitioner#delimiter}.
+	 */
+	public static final String DEFAULT_DELIMITER = String.valueOf('|');
+
+	/**
+	 * The character used to delimit field values in the concatenated partition key string.
+	 */
+	private final String delimiter;
+
+	/**
+	 * A list of field names used to extract the partition key for a record that will be written to
+	 * a Kinesis stream.
+	 */
+	private final List<String> fieldNames;
+
+	/**
+	 * A list of getter functions to dynamically extract the field values for all
+	 * {@link RowDataFieldsKinesisPartitioner#fieldNames} from an input record.
+	 */
+	private final RowData.FieldGetter[] dynamicFieldGetters;
+
+	/**
+	 * A buffer used to accumulate the concatenation of all field values that form the partition
+	 * key.
+	 */
+	private final StringBuilder keyBuffer = new StringBuilder();
+
+	/**
+	 * A prefix of fixed values to be used instead of the corresponding
+	 * {@link RowDataFieldsKinesisPartitioner#dynamicFieldGetters} entries.
+	 */
+	private String[] fixedPrefix = new String[]{};
+
+	/**
+	 * The cumulative length of all values currently stored in the
+	 * {@link RowDataFieldsKinesisPartitioner#fixedPrefix}.
+	 */
+	private int fixedPrefixLength = 0;
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table) {
+		this(table, DEFAULT_DELIMITER);
+	}
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table, String delimiter) {
+		Preconditions.checkNotNull(table, "table");
+		Preconditions.checkNotNull(delimiter, "delimiter");
+		Preconditions.checkArgument(
+			table.isPartitioned(),
+			"Cannot create a RowDataFieldsKinesisPartitioner for a non-partitioned table");
+		Preconditions.checkArgument(
+			table.getPartitionKeys().size() == new HashSet<>(table.getPartitionKeys()).size(),
+			"The sequence of partition keys cannot contain duplicates");
+
+		TableSchema schema = table.getSchema();
+		List<String> schemaFieldsList = Arrays.asList(schema.getFieldNames());
+
+		String[] badKeys = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				boolean hasField = schema.getTableColumn(fieldName).isPresent();
+				return hasField ? Stream.empty() : Stream.of(fieldName);
+			})
+			.toArray(String[]::new);
+		Preconditions.checkArgument(
+			badKeys.length == 0,
+			"The following partition keys are not present in the table: %s",
+			String.join(", ", badKeys));
+
+		this.delimiter = delimiter;
+		this.fieldNames = table.getPartitionKeys();
+		this.dynamicFieldGetters = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				Optional<TableColumn> tableColumn = schema.getTableColumn(fieldName);
+				// convert Optional to Stream to unpack the value in the enclosing flatMap
+				return tableColumn.map(Stream::of).orElseGet(Stream::empty);
+			})
+			.map(column -> {
+				LogicalType fieldType = column.getType().getLogicalType();
+				int fieldPos = schemaFieldsList.indexOf(column.getName());
+				return RowData.createFieldGetter(fieldType, fieldPos);
+			})
+			.toArray(RowData.FieldGetter[]::new);
+	}
+
+	@Override
+	public String getPartitionId(RowData element) {
+		keyBuffer.setLength(fixedPrefixLength); // reset the dynamic part of the key buffer
+		for (int i = fixedPrefix.length; i < dynamicFieldGetters.length; i++) {
+			RowData.FieldGetter getter = dynamicFieldGetters[i];
+			Object fieldValue = getter.getFieldOrNull(element);
+
+			keyBuffer.append(fieldValue);
+			keyBuffer.append(delimiter);
+
+			if (keyBuffer.length() >= MAX_PARTITION_KEY_LENGTH) {
+				break; // stop when the buffer length exceeds the allowed partition key size
+			}
+		}
+
+		// return the accumulated concatenated string trimmed to the max allowed partition key size
+		int length = Math.min(
+			keyBuffer.length() - delimiter.length(),

Review comment:
       why `- delimiter.length()`?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataKinesisDeserializationSchema.java
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A {@link KinesisDeserializationSchema} adaptor for {@link RowData} records that delegates
+ * physical data deserialization to an inner {@link DeserializationSchema} and appends requested
+ * metadata to the end of the deserialized {@link RowData} record.
+ */
+@Internal
+public class RowDataKinesisDeserializationSchema implements KinesisDeserializationSchema<RowData> {
+
+	/** Internal type for enumerating available metadata. */
+	protected enum Metadata {
+		ApproximateArrivalTimestamp(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull()),
+		SequenceNumber(DataTypes.VARCHAR(128).notNull()),
+		ShardId(DataTypes.VARCHAR(128).notNull());
+
+		private final DataType dataType;
+
+		Metadata(DataType dataType) {
+			this.dataType = dataType;
+		}
+
+		public DataType getDataType() {
+			return this.dataType;
+		}
+	}
+
+	private static final long serialVersionUID = 5551095193778230749L;
+
+	/** A {@link DeserializationSchema} to deserialize the physical part of the row. */
+	private final DeserializationSchema<RowData> physicalDeserializer;
+
+	/** The type of the produced {@link RowData} records (physical data with appended metadata]. */
+	private final TypeInformation<RowData> producedTypeInfo;
+
+	/** Metadata fields to be appended to the physical {@link RowData} in the produced records. */
+	private final List<Metadata> requestedMetadataFields;
+
+	public RowDataKinesisDeserializationSchema(
+		DeserializationSchema<RowData> physicalDeserializer,
+		TypeInformation<RowData> producedTypeInfo,
+		List<Metadata> requestedMetadataFields) {
+		this.physicalDeserializer = Preconditions.checkNotNull(physicalDeserializer);
+		this.producedTypeInfo = Preconditions.checkNotNull(producedTypeInfo);
+		this.requestedMetadataFields = Preconditions.checkNotNull(requestedMetadataFields);
+	}
+
+	@Override
+	public void open(DeserializationSchema.InitializationContext context) throws Exception {
+		physicalDeserializer.open(context);
+	}
+
+	@Override
+	public RowData deserialize(
+		byte[] recordValue,
+		String partitionKey,
+		String seqNum,
+		long approxArrivalTimestamp,
+		String stream,
+		String shardId) throws IOException {
+
+		RowData physicalRow = physicalDeserializer.deserialize(recordValue);
+		GenericRowData metadataRow = new GenericRowData(requestedMetadataFields.size());
+
+		for (int i = 0; i < metadataRow.getArity(); i++) {
+			Metadata metadataField = requestedMetadataFields.get(i);
+			if (metadataField == Metadata.ApproximateArrivalTimestamp) {
+				metadataRow.setField(i, TimestampData.fromEpochMillis(approxArrivalTimestamp));
+			} else if (metadataField == Metadata.SequenceNumber) {
+				metadataRow.setField(i, StringData.fromString(seqNum));
+			} else if (metadataField == Metadata.ShardId) {
+				metadataRow.setField(i, StringData.fromString(shardId));
+			} else {
+				String msg = String.format("Unsupported metadata key %s", metadataField);
+				throw new RuntimeException(msg); // should never happen
+			}
+		}
+
+		JoinedRowData joinedRowData = new JoinedRowData(physicalRow, metadataRow);

Review comment:
       I was also thinking about moving this into `table-common`. We can do it but I would rather move it to `org.apache.flink.table.data.utils` because it is not a "core data structure" as the others in the package. In any case we should do this change in a separate commit. The downside of your implementation is that you are creating 2 more objects (row+inner array) but of course this can be neglected.

##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitionerTest.java
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.temporal.ChronoUnit;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataFieldsKinesisPartitioner.MAX_PARTITION_KEY_LENGTH;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for {@link RowDataFieldsKinesisPartitioner}.
+ */
+public class RowDataFieldsKinesisPartitionerTest extends TestLogger {
+
+	/**
+	 * Table name to use for the tests.
+	 */
+	private static final String TABLE_NAME = "click_stream";
+
+	/**
+	 * Table schema to use for the tests.
+	 */
+	private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+		.field("time", DataTypes.TIMESTAMP(3))
+		.field("date", DataTypes.DATE(), "DATE(`time`)")
+		.field("ip", DataTypes.VARCHAR(16))
+		.field("route", DataTypes.STRING())
+		.build();
+
+	/**
+	 * A {@code PARTITION BY(date, ip)} clause to use for the positive tests.
+	 */
+	private static final List<String> PARTITION_BY_DATE_AND_IP = Arrays.asList("date", "ip");
+
+	/**
+	 * A list of field delimiters to use in the tests.
+	 */
+	private static final List<String> FIELD_DELIMITERS = Arrays.asList("", "|", ",", "--");
+
+	/**
+	 * A {@code PARTITION BY(route)} clause to use for the positive tests.
+	 */
+	private static final List<String> PARTITION_BY_ROUTE = Collections.singletonList("route");
+
+	/**
+	 * Some not-so-random {@link LocalDateTime} instances to use for sample {@link RowData}
+	 * elements in the tests.
+	 */
+	private static final List<LocalDateTime> DATE_TIMES = Arrays.asList(
+		LocalDateTime.of(2014, 10, 22, 14, 0),
+		LocalDateTime.of(2015, 12, 14, 14, 0));
+
+	/**
+	 * A default IP to use for sample {@link RowData} elements in the tests.
+	 */
+	private static final String IP = "255.255.255.255";
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodPartitionerWithDynamicPrefix() {
+		CatalogTable table = createTable(defaultTableOptions(), PARTITION_BY_DATE_AND_IP);
+
+		for (String delimiter : FIELD_DELIMITERS) {
+			RowDataFieldsKinesisPartitioner partitioner =
+				new RowDataFieldsKinesisPartitioner(table, delimiter);
+
+			for (LocalDateTime time : DATE_TIMES) {
+				String expectedKey = String.join(delimiter, String.valueOf(days(time)), IP);
+				String actualKey = partitioner.getPartitionId(createElement(time, IP));
+
+				assertEquals(expectedKey, actualKey);
+			}
+		}
+	}
+
+	@Test
+	public void testGoodPartitionerWithDynamicPrefixExceedingMaxLength() {
+		CatalogTable table = createTable(defaultTableOptions(), PARTITION_BY_ROUTE);
+		RowDataFieldsKinesisPartitioner partitioner = new RowDataFieldsKinesisPartitioner(table);
+
+		String ip = "255.255.255.255";
+		String route = "http://www.very-" + repeat("long-", 50) + "address.com/home";
+		String expectedKey = route.substring(0, MAX_PARTITION_KEY_LENGTH);
+
+		for (LocalDateTime time : DATE_TIMES) {
+			String actualKey = partitioner.getPartitionId(createElement(time, ip, route));
+			assertEquals(expectedKey, actualKey);
+		}
+	}
+
+	@Test
+	public void testGoodPartitionerWithStaticPrefix() {
+		CatalogTable table = createTable(defaultTableOptions(), PARTITION_BY_DATE_AND_IP);
+
+		// fixed prefix
+		int date = days(DATE_TIMES.get(0));
+
+		for (String delimiter : FIELD_DELIMITERS) {
+			RowDataFieldsKinesisPartitioner partitioner =
+				new RowDataFieldsKinesisPartitioner(table, delimiter);
+
+			partitioner.setFixedPrefix(new HashMap<String, String>() {{
+				put("date", String.valueOf(date));
+			}});
+
+			for (LocalDateTime time : DATE_TIMES) {
+				String expectedKey = String.join(delimiter, String.valueOf(date), IP);
+				String actualKey = partitioner.getPartitionId(createElement(time, IP));
+
+				assertEquals(expectedKey, actualKey);
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Negative tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testBadPartitionerWithEmptyPrefix() {
+		thrown.expect(IllegalArgumentException.class);
+		thrown.expect(containsCause(new IllegalArgumentException(
+			"Cannot create a RowDataFieldsKinesisPartitioner for a non-partitioned table")));
+
+		CatalogTable table = createTable(defaultTableOptions(), Collections.emptyList());
+		new RowDataFieldsKinesisPartitioner(table);
+	}
+
+	@Test
+	public void testBadPartitionerWithDuplicateFieldNamesInPrefix() {
+		thrown.expect(IllegalArgumentException.class);
+		thrown.expect(containsCause(new IllegalArgumentException(
+			"The sequence of partition keys cannot contain duplicates")));
+
+		CatalogTable table = createTable(defaultTableOptions(), Arrays.asList("ip", "ip"));
+		new RowDataFieldsKinesisPartitioner(table);
+	}
+
+	@Test
+	public void testBadPartitionerWithBadFieldFieldNamesInPrefix() {
+		thrown.expect(IllegalArgumentException.class);
+		thrown.expect(containsCause(new IllegalArgumentException(
+			"The following partition keys are not present in the table: abc")));
+
+		CatalogTable table = createTable(defaultTableOptions(), Arrays.asList("ip", "abc"));
+		new RowDataFieldsKinesisPartitioner(table);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Utilities
+	// --------------------------------------------------------------------------------------------
+
+	private RowData createElement(LocalDateTime time, String ip) {
+		return createElement(time, ip, "https://flink.apache.org/home");
+	}
+
+	private RowData createElement(LocalDateTime time, String ip, String route) {
+		GenericRowData element = new GenericRowData(TABLE_SCHEMA.getFieldCount());
+		element.setField(0, TimestampData.fromLocalDateTime(time));
+		element.setField(1, days(time));
+		element.setField(2, StringData.fromString(ip));
+		element.setField(3, StringData.fromString(route));
+		return element;
+	}
+
+	private int days(LocalDateTime time) {
+		return (int) ChronoUnit.DAYS.between(LocalDate.ofEpochDay(0), time);
+	}
+
+	private CatalogTable createTable(TableOptionsBuilder options, List<String> partitionKeys) {
+		return new CatalogTableImpl(TABLE_SCHEMA, partitionKeys, options.build(), TABLE_NAME);
+	}
+
+	private TableOptionsBuilder defaultTableOptions() {
+		String connector = KinesisDynamicTableFactory.IDENTIFIER;
+		String format = TestFormatFactory.IDENTIFIER;
+		return new TableOptionsBuilder(connector, format)
+			// default table options
+			.withTableOption(KinesisOptions.STREAM, TABLE_NAME)
+			.withTableOption("properties.aws.region", "us-west-2")
+			// default format options
+			.withFormatOption(TestFormatFactory.DELIMITER, ",");
+	}
+
+	/**
+	 * Repeat {@code n} times the string {@code s}.
+	 *
+	 * @param s The string to be repeated.
+	 * @param n The number of times to repeat the string.
+	 *
+	 * @return The repeated string.
+	 */
+	private static String repeat(String s, int n) {

Review comment:
       you can use `org.apache.flink.table.utils.EncodingUtils#repeat(java.lang.String, int)`

##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(

Review comment:
       nit: move this call into a helper method also for the sink

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * <p>A {@link KinesisPartitioner} of {@link RowData} elements that constructs the partition key
+ * from a list of field names.</p>
+ *
+ * <p>The key is constructed by concatenating the string representations of a list of fields
+ * projected from an input element. A fixed prefix can be optionally configured in order to speed
+ * up the key construction process.</p>
+ *
+ * <p>Resulting partition key values are trimmed to the maximum length allowed by Kinesis.</p>
+ */
+@Internal
+public class RowDataFieldsKinesisPartitioner extends KinesisPartitioner<RowData> {
+
+	/**
+	 * Allowed maximum length limit of a partition key.
+	 *
+	 * @link https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#API_PutRecord_RequestSyntax
+	 */
+	public static final int MAX_PARTITION_KEY_LENGTH = 256;
+
+	/**
+	 * Default delimiter for {@link RowDataFieldsKinesisPartitioner#delimiter}.
+	 */
+	public static final String DEFAULT_DELIMITER = String.valueOf('|');
+
+	/**
+	 * The character used to delimit field values in the concatenated partition key string.
+	 */
+	private final String delimiter;
+
+	/**
+	 * A list of field names used to extract the partition key for a record that will be written to
+	 * a Kinesis stream.
+	 */
+	private final List<String> fieldNames;
+
+	/**
+	 * A list of getter functions to dynamically extract the field values for all
+	 * {@link RowDataFieldsKinesisPartitioner#fieldNames} from an input record.
+	 */
+	private final RowData.FieldGetter[] dynamicFieldGetters;
+
+	/**
+	 * A buffer used to accumulate the concatenation of all field values that form the partition
+	 * key.
+	 */
+	private final StringBuilder keyBuffer = new StringBuilder();
+
+	/**
+	 * A prefix of fixed values to be used instead of the corresponding
+	 * {@link RowDataFieldsKinesisPartitioner#dynamicFieldGetters} entries.
+	 */
+	private String[] fixedPrefix = new String[]{};
+
+	/**
+	 * The cumulative length of all values currently stored in the
+	 * {@link RowDataFieldsKinesisPartitioner#fixedPrefix}.
+	 */
+	private int fixedPrefixLength = 0;
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table) {
+		this(table, DEFAULT_DELIMITER);
+	}
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table, String delimiter) {
+		Preconditions.checkNotNull(table, "table");
+		Preconditions.checkNotNull(delimiter, "delimiter");
+		Preconditions.checkArgument(
+			table.isPartitioned(),
+			"Cannot create a RowDataFieldsKinesisPartitioner for a non-partitioned table");
+		Preconditions.checkArgument(
+			table.getPartitionKeys().size() == new HashSet<>(table.getPartitionKeys()).size(),
+			"The sequence of partition keys cannot contain duplicates");
+
+		TableSchema schema = table.getSchema();
+		List<String> schemaFieldsList = Arrays.asList(schema.getFieldNames());
+
+		String[] badKeys = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				boolean hasField = schema.getTableColumn(fieldName).isPresent();
+				return hasField ? Stream.empty() : Stream.of(fieldName);
+			})
+			.toArray(String[]::new);
+		Preconditions.checkArgument(
+			badKeys.length == 0,
+			"The following partition keys are not present in the table: %s",
+			String.join(", ", badKeys));
+
+		this.delimiter = delimiter;
+		this.fieldNames = table.getPartitionKeys();
+		this.dynamicFieldGetters = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				Optional<TableColumn> tableColumn = schema.getTableColumn(fieldName);
+				// convert Optional to Stream to unpack the value in the enclosing flatMap
+				return tableColumn.map(Stream::of).orElseGet(Stream::empty);
+			})
+			.map(column -> {
+				LogicalType fieldType = column.getType().getLogicalType();
+				int fieldPos = schemaFieldsList.indexOf(column.getName());
+				return RowData.createFieldGetter(fieldType, fieldPos);
+			})
+			.toArray(RowData.FieldGetter[]::new);
+	}
+
+	@Override
+	public String getPartitionId(RowData element) {
+		keyBuffer.setLength(fixedPrefixLength); // reset the dynamic part of the key buffer
+		for (int i = fixedPrefix.length; i < dynamicFieldGetters.length; i++) {
+			RowData.FieldGetter getter = dynamicFieldGetters[i];
+			Object fieldValue = getter.getFieldOrNull(element);
+
+			keyBuffer.append(fieldValue);
+			keyBuffer.append(delimiter);
+
+			if (keyBuffer.length() >= MAX_PARTITION_KEY_LENGTH) {
+				break; // stop when the buffer length exceeds the allowed partition key size
+			}
+		}
+
+		// return the accumulated concatenated string trimmed to the max allowed partition key size
+		int length = Math.min(
+			keyBuffer.length() - delimiter.length(),
+			MAX_PARTITION_KEY_LENGTH);
+		return keyBuffer.substring(0, length);
+	}
+
+	/**
+	 * Update the fixed partition key prefix.
+	 *
+	 * @param staticFields An association of (field name, field value) pairs to be used as static
+	 * 	partition key prefix.
+	 */
+	public void setFixedPrefix(Map<String, String> staticFields) {
+		Preconditions.checkArgument(
+			isPartitionPrefix(staticFields.keySet()),
+			String.format(
+				"Not all static field names (%s) are defined as part " +
+					"of the partition key (%s).",
+				String.join(", ", staticFields.keySet()),
+				String.join(", ", fieldNames)
+			));
+
+		// update the fixed prefix and pre-compute its cumulative length
+		fixedPrefix = staticFields.values().toArray(new String[0]);

Review comment:
       don't we assume some implicit order here? the order of partition keys is defined by the `CatalogTable`, static fields must not necessarily be a prefix but can.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * <p>A {@link KinesisPartitioner} of {@link RowData} elements that constructs the partition key
+ * from a list of field names.</p>
+ *
+ * <p>The key is constructed by concatenating the string representations of a list of fields
+ * projected from an input element. A fixed prefix can be optionally configured in order to speed
+ * up the key construction process.</p>
+ *
+ * <p>Resulting partition key values are trimmed to the maximum length allowed by Kinesis.</p>
+ */
+@Internal
+public class RowDataFieldsKinesisPartitioner extends KinesisPartitioner<RowData> {
+
+	/**
+	 * Allowed maximum length limit of a partition key.
+	 *
+	 * @link https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#API_PutRecord_RequestSyntax
+	 */
+	public static final int MAX_PARTITION_KEY_LENGTH = 256;
+
+	/**
+	 * Default delimiter for {@link RowDataFieldsKinesisPartitioner#delimiter}.
+	 */
+	public static final String DEFAULT_DELIMITER = String.valueOf('|');
+
+	/**
+	 * The character used to delimit field values in the concatenated partition key string.
+	 */
+	private final String delimiter;
+
+	/**
+	 * A list of field names used to extract the partition key for a record that will be written to
+	 * a Kinesis stream.
+	 */
+	private final List<String> fieldNames;
+
+	/**
+	 * A list of getter functions to dynamically extract the field values for all
+	 * {@link RowDataFieldsKinesisPartitioner#fieldNames} from an input record.
+	 */
+	private final RowData.FieldGetter[] dynamicFieldGetters;
+
+	/**
+	 * A buffer used to accumulate the concatenation of all field values that form the partition
+	 * key.
+	 */
+	private final StringBuilder keyBuffer = new StringBuilder();
+
+	/**
+	 * A prefix of fixed values to be used instead of the corresponding
+	 * {@link RowDataFieldsKinesisPartitioner#dynamicFieldGetters} entries.
+	 */
+	private String[] fixedPrefix = new String[]{};
+
+	/**
+	 * The cumulative length of all values currently stored in the
+	 * {@link RowDataFieldsKinesisPartitioner#fixedPrefix}.
+	 */
+	private int fixedPrefixLength = 0;
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table) {
+		this(table, DEFAULT_DELIMITER);
+	}
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table, String delimiter) {
+		Preconditions.checkNotNull(table, "table");
+		Preconditions.checkNotNull(delimiter, "delimiter");
+		Preconditions.checkArgument(
+			table.isPartitioned(),
+			"Cannot create a RowDataFieldsKinesisPartitioner for a non-partitioned table");
+		Preconditions.checkArgument(
+			table.getPartitionKeys().size() == new HashSet<>(table.getPartitionKeys()).size(),
+			"The sequence of partition keys cannot contain duplicates");
+
+		TableSchema schema = table.getSchema();
+		List<String> schemaFieldsList = Arrays.asList(schema.getFieldNames());
+
+		String[] badKeys = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				boolean hasField = schema.getTableColumn(fieldName).isPresent();
+				return hasField ? Stream.empty() : Stream.of(fieldName);
+			})
+			.toArray(String[]::new);
+		Preconditions.checkArgument(
+			badKeys.length == 0,
+			"The following partition keys are not present in the table: %s",
+			String.join(", ", badKeys));
+
+		this.delimiter = delimiter;
+		this.fieldNames = table.getPartitionKeys();
+		this.dynamicFieldGetters = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				Optional<TableColumn> tableColumn = schema.getTableColumn(fieldName);
+				// convert Optional to Stream to unpack the value in the enclosing flatMap
+				return tableColumn.map(Stream::of).orElseGet(Stream::empty);
+			})
+			.map(column -> {
+				LogicalType fieldType = column.getType().getLogicalType();
+				int fieldPos = schemaFieldsList.indexOf(column.getName());
+				return RowData.createFieldGetter(fieldType, fieldPos);
+			})
+			.toArray(RowData.FieldGetter[]::new);
+	}
+
+	@Override
+	public String getPartitionId(RowData element) {
+		keyBuffer.setLength(fixedPrefixLength); // reset the dynamic part of the key buffer
+		for (int i = fixedPrefix.length; i < dynamicFieldGetters.length; i++) {
+			RowData.FieldGetter getter = dynamicFieldGetters[i];
+			Object fieldValue = getter.getFieldOrNull(element);
+
+			keyBuffer.append(fieldValue);
+			keyBuffer.append(delimiter);
+
+			if (keyBuffer.length() >= MAX_PARTITION_KEY_LENGTH) {
+				break; // stop when the buffer length exceeds the allowed partition key size
+			}
+		}
+
+		// return the accumulated concatenated string trimmed to the max allowed partition key size
+		int length = Math.min(
+			keyBuffer.length() - delimiter.length(),
+			MAX_PARTITION_KEY_LENGTH);
+		return keyBuffer.substring(0, length);
+	}
+
+	/**
+	 * Update the fixed partition key prefix.
+	 *
+	 * @param staticFields An association of (field name, field value) pairs to be used as static
+	 * 	partition key prefix.
+	 */
+	public void setFixedPrefix(Map<String, String> staticFields) {
+		Preconditions.checkArgument(
+			isPartitionPrefix(staticFields.keySet()),
+			String.format(
+				"Not all static field names (%s) are defined as part " +
+					"of the partition key (%s).",
+				String.join(", ", staticFields.keySet()),
+				String.join(", ", fieldNames)
+			));
+
+		// update the fixed prefix and pre-compute its cumulative length
+		fixedPrefix = staticFields.values().toArray(new String[0]);
+		String fixedPrefixString = Arrays.stream(fixedPrefix)
+			.map(s -> s + delimiter)
+			.collect(Collectors.joining());
+		fixedPrefixLength = fixedPrefixString.length();
+
+		// set a concatenated string of all fixed values as a prefix in the partition key buffer
+		keyBuffer.replace(0, fixedPrefixLength, fixedPrefixString);
+		keyBuffer.setLength(fixedPrefixLength);
+	}
+
+	/**
+	 * Check whether the set of field names in {@code candidatePrefix} forms a valid prefix of the
+	 * list of field names defined in {@link RowDataFieldsKinesisPartitioner#fieldNames} (assuming
+	 * the latter does not contain duplicates).
+	 *
+	 * @param candidatePrefix A set of field names forming a prefix of
+	 *    {@link RowDataFieldsKinesisPartitioner#fieldNames}.
+	 *
+	 * @return true if and only if the {@code candidatePrefix} is actually a prefix.
+	 */
+	private boolean isPartitionPrefix(Set<String> candidatePrefix) {
+		for (String name : fieldNames.subList(0, candidatePrefix.size())) {
+			if (!candidatePrefix.contains(name)) {
+				return false; // name at current position is not in the candidatePrefix
+			}
+		}
+		return true;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Value semantics for equals and hashCode
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+		final RowDataFieldsKinesisPartitioner that = (RowDataFieldsKinesisPartitioner) o;
+		return Objects.equals(this.fieldNames, that.fieldNames) &&
+			Arrays.equals(this.fixedPrefix, that.fixedPrefix);
+	}
+
+	@Override
+	public int hashCode() {
+		return Objects.hash(
+			fieldNames,
+			Arrays.hashCode(fixedPrefix));

Review comment:
       remove `Arrays.hashCode`, shall we add the other fields as well for testing?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/RowDataFieldsKinesisPartitioner.java
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * <p>A {@link KinesisPartitioner} of {@link RowData} elements that constructs the partition key
+ * from a list of field names.</p>
+ *
+ * <p>The key is constructed by concatenating the string representations of a list of fields
+ * projected from an input element. A fixed prefix can be optionally configured in order to speed
+ * up the key construction process.</p>
+ *
+ * <p>Resulting partition key values are trimmed to the maximum length allowed by Kinesis.</p>
+ */
+@Internal
+public class RowDataFieldsKinesisPartitioner extends KinesisPartitioner<RowData> {
+
+	/**
+	 * Allowed maximum length limit of a partition key.
+	 *
+	 * @link https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#API_PutRecord_RequestSyntax
+	 */
+	public static final int MAX_PARTITION_KEY_LENGTH = 256;
+
+	/**
+	 * Default delimiter for {@link RowDataFieldsKinesisPartitioner#delimiter}.
+	 */
+	public static final String DEFAULT_DELIMITER = String.valueOf('|');
+
+	/**
+	 * The character used to delimit field values in the concatenated partition key string.
+	 */
+	private final String delimiter;
+
+	/**
+	 * A list of field names used to extract the partition key for a record that will be written to
+	 * a Kinesis stream.
+	 */
+	private final List<String> fieldNames;
+
+	/**
+	 * A list of getter functions to dynamically extract the field values for all
+	 * {@link RowDataFieldsKinesisPartitioner#fieldNames} from an input record.
+	 */
+	private final RowData.FieldGetter[] dynamicFieldGetters;
+
+	/**
+	 * A buffer used to accumulate the concatenation of all field values that form the partition
+	 * key.
+	 */
+	private final StringBuilder keyBuffer = new StringBuilder();
+
+	/**
+	 * A prefix of fixed values to be used instead of the corresponding
+	 * {@link RowDataFieldsKinesisPartitioner#dynamicFieldGetters} entries.
+	 */
+	private String[] fixedPrefix = new String[]{};
+
+	/**
+	 * The cumulative length of all values currently stored in the
+	 * {@link RowDataFieldsKinesisPartitioner#fixedPrefix}.
+	 */
+	private int fixedPrefixLength = 0;
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table) {
+		this(table, DEFAULT_DELIMITER);
+	}
+
+	public RowDataFieldsKinesisPartitioner(CatalogTable table, String delimiter) {
+		Preconditions.checkNotNull(table, "table");
+		Preconditions.checkNotNull(delimiter, "delimiter");
+		Preconditions.checkArgument(
+			table.isPartitioned(),
+			"Cannot create a RowDataFieldsKinesisPartitioner for a non-partitioned table");
+		Preconditions.checkArgument(
+			table.getPartitionKeys().size() == new HashSet<>(table.getPartitionKeys()).size(),
+			"The sequence of partition keys cannot contain duplicates");
+
+		TableSchema schema = table.getSchema();
+		List<String> schemaFieldsList = Arrays.asList(schema.getFieldNames());
+
+		String[] badKeys = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				boolean hasField = schema.getTableColumn(fieldName).isPresent();
+				return hasField ? Stream.empty() : Stream.of(fieldName);
+			})
+			.toArray(String[]::new);
+		Preconditions.checkArgument(
+			badKeys.length == 0,
+			"The following partition keys are not present in the table: %s",
+			String.join(", ", badKeys));
+
+		this.delimiter = delimiter;
+		this.fieldNames = table.getPartitionKeys();
+		this.dynamicFieldGetters = table.getPartitionKeys().stream()
+			.flatMap(fieldName -> {
+				Optional<TableColumn> tableColumn = schema.getTableColumn(fieldName);
+				// convert Optional to Stream to unpack the value in the enclosing flatMap
+				return tableColumn.map(Stream::of).orElseGet(Stream::empty);
+			})
+			.map(column -> {
+				LogicalType fieldType = column.getType().getLogicalType();
+				int fieldPos = schemaFieldsList.indexOf(column.getName());
+				return RowData.createFieldGetter(fieldType, fieldPos);
+			})
+			.toArray(RowData.FieldGetter[]::new);
+	}
+
+	@Override
+	public String getPartitionId(RowData element) {
+		keyBuffer.setLength(fixedPrefixLength); // reset the dynamic part of the key buffer
+		for (int i = fixedPrefix.length; i < dynamicFieldGetters.length; i++) {
+			RowData.FieldGetter getter = dynamicFieldGetters[i];
+			Object fieldValue = getter.getFieldOrNull(element);
+
+			keyBuffer.append(fieldValue);
+			keyBuffer.append(delimiter);
+
+			if (keyBuffer.length() >= MAX_PARTITION_KEY_LENGTH) {
+				break; // stop when the buffer length exceeds the allowed partition key size
+			}
+		}
+
+		// return the accumulated concatenated string trimmed to the max allowed partition key size
+		int length = Math.min(
+			keyBuffer.length() - delimiter.length(),
+			MAX_PARTITION_KEY_LENGTH);
+		return keyBuffer.substring(0, length);
+	}
+
+	/**
+	 * Update the fixed partition key prefix.
+	 *
+	 * @param staticFields An association of (field name, field value) pairs to be used as static
+	 * 	partition key prefix.
+	 */
+	public void setFixedPrefix(Map<String, String> staticFields) {
+		Preconditions.checkArgument(
+			isPartitionPrefix(staticFields.keySet()),
+			String.format(
+				"Not all static field names (%s) are defined as part " +
+					"of the partition key (%s).",
+				String.join(", ", staticFields.keySet()),
+				String.join(", ", fieldNames)
+			));
+
+		// update the fixed prefix and pre-compute its cumulative length
+		fixedPrefix = staticFields.values().toArray(new String[0]);

Review comment:
       the exception above could occur quite often, instead we should make static partitions possible even if they are not a prefix




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r519167893



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/JoinedRowData.java
##########
@@ -16,9 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.data;
+package org.apache.flink.table.data.utils;

Review comment:
       I don't have a strong opinion as well. My argument of keeping the class in `org.apache.flink.table.data` was maintaining backwards-compatability, but based on the established guarantees and expectations this does not seem essential.
   
   I will add a hotfix commit which adds `hashCode` and `equals` before changing the package. Feel free to keep or drop the package change commit when you are merging the PR. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723341214


   @twalthr for the next iteration please check the following two `fixup!` commits:
   
   - 96c368253be8bcd395ed05f5f99b3f07cbca4f52, which adds support for non-prefix static keys in `RowDataFieldsKinesisPartitioner`. Test coverage is added in
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticPrefix`,
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticSuffix`, and
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticInfix`.
   - c763dced7f725f78ddf9e1c0b83b29d7769b00a5, which reworks option names and mapping [as discussed in the GoogleDoc](https://docs.google.com/document/d/1Gchs3gK3nnWyodZMYWyK2q0APRQf1gofsTxKBzwzPLQ/edit?usp=sharing).
     - `KinesisOptions#getConsumerProperties` implements the mapping of the properties for the Table source.
     - `KinesisOptions#getProducerProperties` implements the mapping of the properties for the Table sink.
     - Test coverage is added to the happy path tests in `KinesisDynamicTableFactoryTest`.
   
   ### TODOs:
   
   - [ ] Create a follow-up JIRA in oder to add a `KinesisTableITCase`.
   - [ ] Create a follow-up JIRA in order to add a `flink-sql-connector-kinesis` package. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-723341214


   @twalthr for the next review please check:
   
   - The diff of 96c368253be8bcd395ed05f5f99b3f07cbca4f52, which adds support for non-prefix static keys in `RowDataFieldsKinesisPartitioner`. Test coverage is added in
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticPrefix`,
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticSuffix`, and
     - `RowDataFieldsKinesisPartitionerTest#testGoodPartitionerWithStaticInfix`.
   - The diff of c763dced7f725f78ddf9e1c0b83b29d7769b00a5, which reworks option names and mapping [as discussed in the GoogleDoc](https://docs.google.com/document/d/1Gchs3gK3nnWyodZMYWyK2q0APRQf1gofsTxKBzwzPLQ/edit?usp=sharing).
     - The `KinesisOptions#getConsumerProperties` implements the mapping of the properties for the Table source.
     - The `KinesisOptions#getProducerProperties` implements the mapping of the properties for the Table sink.
     - Test coverage is added to the happy path tests in `KinesisDynamicTableFactoryTest`.
   
   ### TODOs:
   
   - [ ] Create a follow-up JIRA in oder to add a `KinesisTableITCase`.
   - [ ] Create a follow-up JIRA in order to add a `flink-sql-connector-kinesis` package. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] whummer commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
whummer commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-721820441


   Many thanks @aalexandrov for this implementation - really exciting to see this feature being added to Flink! 🚀 
   
   > I was hoping we have some dummy Kinesis infrastructure already to test our connectors locally. 
   
   @twalthr Perhaps [LocalStack](https://github.com/localstack/localstack) could help out here? It supports an emulated version of Kinesis (and other AWS APIs) running locally, packaged as a Docker image. (If anything should be missing for testing this, we'd be very happy to add any required functionality in LocalStack.) Thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016) 
   * f09502d6fa6f4f7122359956293194455decb303 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6ffeeef6d919ed2691e41ba5130f98292ab8379f Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535) 
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * c195c6d2c594e176fdbc038ee9672a90dd10ef80 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517301382



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] aalexandrov commented on a change in pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517277900



##########
File path: flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+	private static final String SOURCE_STREAM = "sourceStream";
+	private static final String SINK_STREAM = "targetStream";
+
+	private static final String SOURCE_TABLE = "sourceTable";
+	private static final String SINK_TABLE = "sinkTable";
+
+	private static final Properties KINESIS_PROPERTIES = new Properties() {{
+		setProperty("aws.region", "us-west-2");
+	}};
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	// --------------------------------------------------------------------------------------------
+	// Positive tests
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testGoodTableSource() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		// verify that the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource);
+
+		// verify that the copy of the constructed DynamicTableSink is as expected
+		assertEquals(expectedSource, actualSource.copy());
+
+		// verify produced sink
+		ScanTableSource.ScanRuntimeProvider functionProvider =
+			actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+		SourceFunction<RowData> sourceFunction =
+			as(functionProvider, SourceFunctionProvider.class).createSourceFunction();
+		assertThat(sourceFunction, instanceOf(FlinkKinesisConsumer.class));
+	}
+
+	@Test
+	public void testGoodTableSourceWithMetadataFields() {
+		TableSchema sourceSchema = defaultSourceSchema().build();
+		Map<String, String> sourceOptions = defaultSourceOptions().build();
+
+		// Construct actual DynamicTableSource using FactoryUtil
+		KinesisDynamicSource actualSource = (KinesisDynamicSource)
+			FactoryUtil.createTableSource(
+				null,
+				ObjectIdentifier.of("default", "default", SOURCE_TABLE),
+				createSourceTable(sourceSchema, sourceOptions, Collections.emptyList()),
+				new Configuration(),
+				Thread.currentThread().getContextClassLoader(),
+				false);
+
+		// Construct expected DynamicTableSink using factory under test
+		KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+			sourceSchema.toPhysicalRowDataType(),
+			SOURCE_STREAM,
+			KINESIS_PROPERTIES,
+			new TestFormatFactory.DecodingFormatMock(",", true));
+
+		List<String> metadataKeys = Arrays.asList("ShardId", "ApproximateArrivalTimestamp");
+		DataType producedDataType = getProducedType(sourceSchema, Metadata.values());
+
+		expectedSource.applyReadableMetadata(metadataKeys, producedDataType);

Review comment:
       > we should test the contents of mutable member variables 
   
   These are tested by the following `assertEquals(expectedSource, actualSource);` call, where `KinesisDynamicSource::equals` is
   
   ```
   @Override
   public boolean equals(Object o) {
   	if (this == o) {
   		return true;
   	}
   	if (o == null || getClass() != o.getClass()) {
   		return false;
   	}
   	KinesisDynamicSource that = (KinesisDynamicSource) o;
   	return Objects.equals(producedDataType, that.producedDataType) &&
   		Objects.equals(requestedMetadataFields, that.requestedMetadataFields) &&
   		Objects.equals(physicalDataType, that.physicalDataType) &&
   		Objects.equals(stream, that.stream) &&
   		Objects.equals(consumerProperties, that.consumerProperties) &&
   		Objects.equals(decodingFormat, that.decodingFormat);
   }
   ```
   
   This goes somewhat [against the coding guideline](https://flink.apache.org/contributing/code-style-and-quality-java.html#equals--hashcode), but it is consistent with all other connectors (including `KafkaDynamicSource`). For me this is indicative of a broader issue with the ability interfaces. IMHO calls of ability interface methods should return modified copies of the original source/sink node as opposed to mutating that node (this is compatible with the current design of these classes as immutable factories).  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong commented on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
wuchong commented on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-722186685


   Thanks @aalexandrov  for the great work! 
   
   I agree with @twalthr , it would be great if we can have a separate discussion on the property naming. 
   Would be better to move this discussion to JIRA issue or Google doc or dev ML. 
   We can list all the property keys there, and reach a consensus quickly for all of them. 
   This would be easier to track than mixing in the long pull request reviews. 
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13770: [FLINK-18858][connector-kinesis] Add Kinesis sources and sinks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13770:
URL: https://github.com/apache/flink/pull/13770#issuecomment-715323245


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8191",
       "triggerID" : "88ae6048ac2efeb04ebc076a93da3667c013ccf6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8205",
       "triggerID" : "f1f444a6be029f1558052f93e2a3571aa4ab3613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8326",
       "triggerID" : "a5ce7bc8a6c533b701598debf84e6dc2287f152a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8409",
       "triggerID" : "c9034134f3319c2ac5856d3e41c133148eb39e71",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8521",
       "triggerID" : "a23ca89bdf7d92515892df5ea5ac765f83e86856",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8535",
       "triggerID" : "6ffeeef6d919ed2691e41ba5130f98292ab8379f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c1af295630bc8064312d310846613a8a3b23bbca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8539",
       "triggerID" : "c195c6d2c594e176fdbc038ee9672a90dd10ef80",
       "triggerType" : "PUSH"
     }, {
       "hash" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8711",
       "triggerID" : "379ea4b3bbe5423e57a293dea90768a3e37b923a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8803",
       "triggerID" : "4c062f14c94f4ad5a2720b64f36759f01621d49b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8810",
       "triggerID" : "115d8d0f08bbdee1ca6becd32a0ca8698b486d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8908",
       "triggerID" : "8c5514fdea5224ecd5b2cbc5b95eb6ce22e8289d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8917",
       "triggerID" : "9b9e9423b9708b6bf50c08f4a0d03fb2e88637c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9001",
       "triggerID" : "88abeded29996fbeb101ec7e5498f1c33ff4ae67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016",
       "triggerID" : "c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f09502d6fa6f4f7122359956293194455decb303",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200",
       "triggerID" : "f09502d6fa6f4f7122359956293194455decb303",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209",
       "triggerID" : "a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c1af295630bc8064312d310846613a8a3b23bbca UNKNOWN
   * c92668fdc1ca6d180a0ad669e6cb3d44fa72a5f0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9016) 
   * f09502d6fa6f4f7122359956293194455decb303 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9200) 
   * a4f4adcfa0b461caba1e9e9bfdee15a8e8446b69 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9209) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org