Skip to content

Commit

Permalink
[FLINK-31987] Implement Table API support
Browse files Browse the repository at this point in the history
  • Loading branch information
darenwkt committed Jun 20, 2023
1 parent 1006e6a commit 756f45f
Show file tree
Hide file tree
Showing 19 changed files with 808 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@
import org.apache.flink.api.common.time.Deadline;
import org.apache.flink.connector.aws.testutils.AWSServicesTestUtils;
import org.apache.flink.connector.aws.util.AWSGeneralUtil;
import org.apache.flink.connector.kinesis.testutils.KinesaliteContainer;
import org.apache.flink.connector.testframe.container.FlinkContainers;
import org.apache.flink.connector.testframe.container.TestcontainersSettings;
import org.apache.flink.connectors.kinesis.testutils.KinesaliteContainer;
import org.apache.flink.test.resources.ResourceTestUtils;
import org.apache.flink.test.util.SQLJobSubmission;
import org.apache.flink.util.DockerImageVersions;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@
package org.apache.flink.streaming.kinesis.test;

import org.apache.flink.api.common.time.Deadline;
import org.apache.flink.connector.kinesis.testutils.KinesaliteContainer;
import org.apache.flink.connector.testframe.container.FlinkContainers;
import org.apache.flink.connector.testframe.container.TestcontainersSettings;
import org.apache.flink.connectors.kinesis.testutils.KinesaliteContainer;
import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisPubsubClient;
import org.apache.flink.streaming.kinesis.test.model.Order;
import org.apache.flink.test.resources.ResourceTestUtils;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.flink.glue.schema.registry.test;

import org.apache.flink.api.common.time.Deadline;
import org.apache.flink.connectors.kinesis.testutils.KinesaliteContainer;
import org.apache.flink.connector.kinesis.testutils.KinesaliteContainer;
import org.apache.flink.formats.avro.glue.schema.registry.GlueSchemaRegistryAvroDeserializationSchema;
import org.apache.flink.formats.avro.glue.schema.registry.GlueSchemaRegistryAvroSerializationSchema;
import org.apache.flink.streaming.api.datastream.DataStream;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.flink.glue.schema.registry.test.json;

import org.apache.flink.api.common.time.Deadline;
import org.apache.flink.connectors.kinesis.testutils.KinesaliteContainer;
import org.apache.flink.connector.kinesis.testutils.KinesaliteContainer;
import org.apache.flink.formats.json.glue.schema.registry.GlueSchemaRegistryJsonDeserializationSchema;
import org.apache.flink.formats.json.glue.schema.registry.GlueSchemaRegistryJsonSerializationSchema;
import org.apache.flink.streaming.api.datastream.DataStream;
Expand Down
9 changes: 9 additions & 0 deletions flink-connector-aws-kinesis-streams/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,10 @@ under the License.
<groupId>software.amazon.awssdk</groupId>
<artifactId>kinesis</artifactId>
</dependency>
<dependency>
<groupId>software.amazon.awssdk</groupId>
<artifactId>aws-core</artifactId>
</dependency>

<dependency>
<groupId>software.amazon.awssdk</groupId>
Expand All @@ -69,6 +73,11 @@ under the License.
<artifactId>flink-table-common</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge</artifactId>
<version>${flink.version}</version>
</dependency>

<!-- Test dependencies -->
<dependency>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,20 @@

import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.aws.config.AWSConfigConstants;
import org.apache.flink.util.Preconditions;

import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.Properties;

import static org.apache.flink.connector.aws.util.AWSGeneralUtil.validateAwsConfiguration;
import static org.apache.flink.connector.base.table.util.ConfigurationValidatorUtil.validateOptionalDateProperty;
import static org.apache.flink.connector.base.table.util.ConfigurationValidatorUtil.validateOptionalPositiveLongProperty;
import static org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_INITIAL_TIMESTAMP;
import static org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT;
import static org.apache.flink.util.Preconditions.checkNotNull;

/** Utility functions to use with {@link KinesisStreamsSourceConfigConstants}. */
@Internal
Expand Down Expand Up @@ -58,4 +64,51 @@ public static Date parseStreamTimestampStartingPosition(final Configuration sour
return new Date((long) (Double.parseDouble(timestamp) * 1000));
}
}

/**
* Validate configuration properties for {@link
* org.apache.flink.connector.kinesis.source.KinesisStreamsSource}.
*/
public static void validateStreamSourceConfiguration(Configuration config) {
checkNotNull(config, "config can not be null");

Properties consumerProperties = new Properties();
config.addAllToProperties(consumerProperties);
validateAwsConfiguration(consumerProperties);

if (!(config.containsKey(AWSConfigConstants.AWS_REGION)
|| config.containsKey(AWSConfigConstants.AWS_ENDPOINT))) {
// per validation in AwsClientBuilder
throw new IllegalArgumentException(
String.format(
"For KinesisStreamsSource AWS region ('%s') and/or AWS endpoint ('%s') must be set in the config.",
AWSConfigConstants.AWS_REGION, AWSConfigConstants.AWS_ENDPOINT));
}

if (config.contains(KinesisStreamsSourceConfigConstants.STREAM_INITIAL_POSITION)) {
KinesisStreamsSourceConfigConstants.InitialPosition initPosType =
config.get(KinesisStreamsSourceConfigConstants.STREAM_INITIAL_POSITION);

// specified initial timestamp in stream when using AT_TIMESTAMP
if (initPosType == KinesisStreamsSourceConfigConstants.InitialPosition.AT_TIMESTAMP) {
if (!config.contains(STREAM_INITIAL_TIMESTAMP)) {
throw new IllegalArgumentException(
"Please set value for initial timestamp ('"
+ STREAM_INITIAL_TIMESTAMP
+ "') when using AT_TIMESTAMP initial position.");
}
validateOptionalDateProperty(
consumerProperties,
String.valueOf(STREAM_INITIAL_TIMESTAMP),
config.getString(STREAM_TIMESTAMP_DATE_FORMAT),
"Invalid value given for initial timestamp for AT_TIMESTAMP initial position in stream. "
+ "Must be a valid format: yyyy-MM-dd'T'HH:mm:ss.SSSXXX or non-negative double value. For example, 2016-04-04T19:58:46.480-00:00 or 1459799926.480 .");
}
}

validateOptionalPositiveLongProperty(
consumerProperties,
String.valueOf(KinesisStreamsSourceConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS),
"Invalid value given for shard discovery sleep interval in milliseconds. Must be a valid non-negative long value.");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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.connector.kinesis.source.model;

import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.types.DataType;

import java.util.Arrays;
import java.util.Objects;

/** Internal type for enumerating available metadata. */
public enum Metadata {
Timestamp("timestamp", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull()),
SequenceNumber("sequence-number", DataTypes.VARCHAR(128).notNull()),
ShardId("shard-id", DataTypes.VARCHAR(128).notNull());

private final String fieldName;
private final DataType dataType;

Metadata(String fieldName, DataType dataType) {
this.fieldName = fieldName;
this.dataType = dataType;
}

public String getFieldName() {
return this.fieldName;
}

public DataType getDataType() {
return this.dataType;
}

public static Metadata of(String fieldName) {
return Arrays.stream(Metadata.values())
.filter(m -> Objects.equals(m.fieldName, fieldName))
.findFirst()
.orElseThrow(
() -> {
String msg =
"Cannot find Metadata instance for field name '"
+ fieldName
+ "'";
return new IllegalArgumentException(msg);
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,12 @@ public class KinesisConnectorOptions extends AsyncSinkConnectorOptions {
.noDefaultValue()
.withDescription("Name of the Kinesis stream backing this table.");

public static final ConfigOption<String> STREAM_ARN =
ConfigOptions.key("stream.arn")
.stringType()
.noDefaultValue()
.withDescription("ARN of the Kinesis stream backing this table.");

public static final ConfigOption<String> AWS_REGION =
ConfigOptions.key("aws.region")
.stringType()
Expand Down
Loading

0 comments on commit 756f45f

Please sign in to comment.