Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-16124] YAML-ized Kinesis ingress / egresses #64

Closed
wants to merge 11 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -111,9 +111,6 @@ public static Map<String, String> propertiesAt(JsonNode node, JsonPointer pointe
throw new WrongTypeException(pointer, "not a key-value list");
}
Map.Entry<String, JsonNode> field = fields.next();
if (!field.getValue().isTextual()) {
throw new WrongTypeException(pointer, "not a key-value pair at " + field);
}
properties.put(field.getKey(), field.getValue().asText());
}
return properties;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@
import org.apache.flink.statefun.flink.core.protorouter.AutoRoutableProtobufRouter;
import org.apache.flink.statefun.flink.core.protorouter.ProtobufRouter;
import org.apache.flink.statefun.flink.io.kafka.ProtobufKafkaIngressTypes;
import org.apache.flink.statefun.flink.io.kinesis.PolyglotKinesisIOTypes;
import org.apache.flink.statefun.flink.io.spi.JsonEgressSpec;
import org.apache.flink.statefun.flink.io.spi.JsonIngressSpec;
import org.apache.flink.statefun.sdk.EgressType;
Expand Down Expand Up @@ -139,7 +140,7 @@ private void configureIngress(Binder binder, Iterable<? extends JsonNode> ingres
JsonIngressSpec<Message> ingressSpec = new JsonIngressSpec<>(type, id, ingress);
binder.bindIngress(ingressSpec);

if (type.equals(ProtobufKafkaIngressTypes.ROUTABLE_PROTOBUF_KAFKA_INGRESS_TYPE)) {
if (isAutoRoutableIngress(type)) {
binder.bindIngressRouter(id, new AutoRoutableProtobufRouter());
}
}
Expand Down Expand Up @@ -170,6 +171,11 @@ private static IngressIdentifier<Message> ingressId(JsonNode ingress) {
return new IngressIdentifier<>(Message.class, nn.namespace(), nn.name());
}

private static boolean isAutoRoutableIngress(IngressType ingressType) {
return ingressType.equals(ProtobufKafkaIngressTypes.ROUTABLE_PROTOBUF_KAFKA_INGRESS_TYPE)
|| ingressType.equals(PolyglotKinesisIOTypes.ROUTABLE_PROTOBUF_KINESIS_INGRESS_TYPE);
}

// ----------------------------------------------------------------------------------------------------------
// Egresses
// ----------------------------------------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,16 +24,14 @@
import org.apache.flink.statefun.flink.io.generated.AutoRoutable;
import org.apache.flink.statefun.flink.io.generated.RoutingConfig;
import org.apache.flink.statefun.flink.io.generated.TargetFunctionType;
import org.apache.flink.statefun.flink.io.kafka.ProtobufKafkaIngressTypes;
import org.apache.flink.statefun.sdk.FunctionType;
import org.apache.flink.statefun.sdk.io.Router;

/**
* A {@link Router} that recognizes messages of type {@link AutoRoutable}.
*
* <p>For each incoming {@code AutoRoutable}, this router forwards the wrapped payload to the
* configured target addresses as a Protobuf {@link Any} message. This should only be attached to
* ingress types of {@link ProtobufKafkaIngressTypes#ROUTABLE_PROTOBUF_KAFKA_INGRESS_TYPE}.
* configured target addresses as a Protobuf {@link Any} message.
*/
public final class AutoRoutableProtobufRouter implements Router<Message> {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

import com.google.auto.service.AutoService;
import java.util.Map;
import org.apache.flink.statefun.flink.io.kinesis.polyglot.GenericKinesisSinkProvider;
import org.apache.flink.statefun.flink.io.kinesis.polyglot.RoutableProtobufKinesisSourceProvider;
import org.apache.flink.statefun.flink.io.spi.FlinkIoModule;
import org.apache.flink.statefun.sdk.kinesis.KinesisIOTypes;

Expand All @@ -29,5 +31,10 @@ public final class KinesisFlinkIOModule implements FlinkIoModule {
public void configure(Map<String, String> globalConfiguration, Binder binder) {
binder.bindSourceProvider(KinesisIOTypes.UNIVERSAL_INGRESS_TYPE, new KinesisSourceProvider());
binder.bindSinkProvider(KinesisIOTypes.UNIVERSAL_EGRESS_TYPE, new KinesisSinkProvider());
binder.bindSourceProvider(
PolyglotKinesisIOTypes.ROUTABLE_PROTOBUF_KINESIS_INGRESS_TYPE,
new RoutableProtobufKinesisSourceProvider());
binder.bindSinkProvider(
PolyglotKinesisIOTypes.GENERIC_KINESIS_EGRESS_TYPE, new GenericKinesisSinkProvider());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;

final class KinesisSinkProvider implements SinkProvider {
public final class KinesisSinkProvider implements SinkProvider {

@Override
public <T> SinkFunction<T> forSpec(EgressSpec<T> spec) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,8 @@
import java.time.ZonedDateTime;
import java.time.format.DateTimeFormatter;
import java.util.Properties;
import org.apache.flink.statefun.flink.io.common.ReflectionUtil;
import org.apache.flink.statefun.flink.io.spi.SourceProvider;
import org.apache.flink.statefun.sdk.io.IngressSpec;
import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressDeserializer;
import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressSpec;
import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressStartupPosition;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
Expand All @@ -32,7 +30,7 @@
import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;

final class KinesisSourceProvider implements SourceProvider {
public final class KinesisSourceProvider implements SourceProvider {

@Override
public <T> SourceFunction<T> forSpec(IngressSpec<T> spec) {
Expand All @@ -56,9 +54,7 @@ private static <T> KinesisIngressSpec<T> asKinesisSpec(IngressSpec<T> spec) {

private static <T> KinesisDeserializationSchema<T> deserializationSchemaFromSpec(
KinesisIngressSpec<T> spec) {
KinesisIngressDeserializer<T> ingressDeserializer =
ReflectionUtil.instantiate(spec.deserializerClass());
return new KinesisDeserializationSchemaDelegate<>(ingressDeserializer);
return new KinesisDeserializationSchemaDelegate<>(spec.deserializer());
}

private static Properties propertiesFromSpec(KinesisIngressSpec<?> spec) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
/*
* 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.statefun.flink.io.kinesis.polyglot;

import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
import org.apache.flink.statefun.flink.common.json.Selectors;
import org.apache.flink.statefun.sdk.kinesis.auth.AwsCredentials;
import org.apache.flink.statefun.sdk.kinesis.auth.AwsRegion;

final class AwsAuthSpecJsonParser {

private AwsAuthSpecJsonParser() {}

private static final JsonPointer AWS_REGION_POINTER = JsonPointer.compile("/awsRegion");
private static final JsonPointer AWS_CREDENTIALS_POINTER = JsonPointer.compile("/awsCredentials");

private static final class Region {
private static final String DEFAULT_TYPE = "default";
private static final String SPECIFIED_ID_TYPE = "specific";
private static final String CUSTOM_ENDPOINT_TYPE = "custom-endpoint";

private static final JsonPointer TYPE_POINTER = JsonPointer.compile("/type");
private static final JsonPointer ID_POINTER = JsonPointer.compile("/id");
private static final JsonPointer ENDPOINT_POINTER = JsonPointer.compile("/endpoint");
}

private static final class Credentials {
private static final String DEFAULT_TYPE = "default";
private static final String BASIC_TYPE = "basic";
private static final String PROFILE_TYPE = "profile";

private static final JsonPointer TYPE_POINTER = JsonPointer.compile("/type");
private static final JsonPointer ACCESS_KEY_ID_POINTER = JsonPointer.compile("/accessKeyId");
private static final JsonPointer SECRET_ACCESS_KEY_POINTER =
JsonPointer.compile("/secretAccessKey");
private static final JsonPointer PROFILE_NAME_POINTER = JsonPointer.compile("/profileName");
private static final JsonPointer PROFILE_PATH_POINTER = JsonPointer.compile("/profilePath");
}

static Optional<AwsRegion> optionalAwsRegion(JsonNode specNode) {
final JsonNode awsRegionSpecNode = specNode.at(AWS_REGION_POINTER);
if (awsRegionSpecNode.isMissingNode()) {
return Optional.empty();
}

final String type = Selectors.textAt(awsRegionSpecNode, Region.TYPE_POINTER);
switch (type) {
case Region.DEFAULT_TYPE:
return Optional.of(AwsRegion.fromDefaultProviderChain());
case Region.SPECIFIED_ID_TYPE:
return Optional.of(AwsRegion.ofId(Selectors.textAt(awsRegionSpecNode, Region.ID_POINTER)));
case Region.CUSTOM_ENDPOINT_TYPE:
return Optional.of(
AwsRegion.ofCustomEndpoint(
Selectors.textAt(awsRegionSpecNode, Region.ENDPOINT_POINTER),
Selectors.textAt(awsRegionSpecNode, Region.ID_POINTER)));
default:
final List<String> validValues =
Arrays.asList(
Region.DEFAULT_TYPE, Region.SPECIFIED_ID_TYPE, Region.CUSTOM_ENDPOINT_TYPE);
throw new IllegalArgumentException(
"Invalid AWS region type: "
+ type
+ "; valid values are ["
+ String.join(", ", validValues)
+ "]");
}
}

static Optional<AwsCredentials> optionalAwsCredentials(JsonNode specNode) {
final JsonNode awsCredentialsSpecNode = specNode.at(AWS_CREDENTIALS_POINTER);
if (awsCredentialsSpecNode.isMissingNode()) {
return Optional.empty();
}

final String type = Selectors.textAt(awsCredentialsSpecNode, Credentials.TYPE_POINTER);
switch (type) {
case Credentials.DEFAULT_TYPE:
return Optional.of(AwsCredentials.fromDefaultProviderChain());
case Credentials.BASIC_TYPE:
return Optional.of(
AwsCredentials.basic(
Selectors.textAt(awsCredentialsSpecNode, Credentials.ACCESS_KEY_ID_POINTER),
Selectors.textAt(awsCredentialsSpecNode, Credentials.SECRET_ACCESS_KEY_POINTER)));
case Credentials.PROFILE_TYPE:
final Optional<String> path =
Selectors.optionalTextAt(awsCredentialsSpecNode, Credentials.PROFILE_PATH_POINTER);
if (path.isPresent()) {
return Optional.of(
AwsCredentials.profile(
Selectors.textAt(awsCredentialsSpecNode, Credentials.PROFILE_NAME_POINTER),
path.get()));
} else {
return Optional.of(
AwsCredentials.profile(
Selectors.textAt(awsCredentialsSpecNode, Credentials.PROFILE_NAME_POINTER)));
}
default:
final List<String> validValues =
Arrays.asList(
Credentials.DEFAULT_TYPE, Credentials.BASIC_TYPE, Credentials.PROFILE_TYPE);
throw new IllegalArgumentException(
"Invalid AWS credential type: "
+ type
+ "; valid values are ["
+ String.join(", ", validValues)
+ "]");
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.statefun.flink.io.kinesis.polyglot;

import com.google.protobuf.Any;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.flink.statefun.flink.io.generated.KinesisEgressRecord;
import org.apache.flink.statefun.sdk.kinesis.egress.EgressRecord;
import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressSerializer;

public final class GenericKinesisEgressSerializer implements KinesisEgressSerializer<Any> {

private static final long serialVersionUID = 1L;

@Override
public EgressRecord serialize(Any value) {
final KinesisEgressRecord kinesisEgressRecord = asKinesisEgressRecord(value);
return EgressRecord.newBuilder()
.withData(kinesisEgressRecord.getValueBytes().toByteArray())
.withStream(kinesisEgressRecord.getStream())
.withPartitionKey(kinesisEgressRecord.getPartitionKey())
.withExplicitHashKey(kinesisEgressRecord.getExplicitHashKey())
.build();
}

private static KinesisEgressRecord asKinesisEgressRecord(Any message) {
if (!message.is(KinesisEgressRecord.class)) {
throw new IllegalStateException(
"The generic Kinesis egress expects only messages of type "
+ KinesisEgressRecord.class.getName());
}
try {
return message.unpack(KinesisEgressRecord.class);
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException(
"Unable to unpack message as a " + KinesisEgressRecord.class.getName(), e);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
/*
* 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.statefun.flink.io.kinesis.polyglot;

import static org.apache.flink.statefun.flink.io.kinesis.polyglot.AwsAuthSpecJsonParser.optionalAwsCredentials;
import static org.apache.flink.statefun.flink.io.kinesis.polyglot.AwsAuthSpecJsonParser.optionalAwsRegion;
import static org.apache.flink.statefun.flink.io.kinesis.polyglot.KinesisEgressSpecJsonParser.optionalMaxOutstandingRecords;

import com.google.protobuf.Any;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
import org.apache.flink.statefun.flink.io.kinesis.KinesisSinkProvider;
import org.apache.flink.statefun.flink.io.spi.JsonEgressSpec;
import org.apache.flink.statefun.flink.io.spi.SinkProvider;
import org.apache.flink.statefun.sdk.io.EgressIdentifier;
import org.apache.flink.statefun.sdk.io.EgressSpec;
import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressBuilder;
import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressSpec;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;

public final class GenericKinesisSinkProvider implements SinkProvider {

private final KinesisSinkProvider delegateProvider = new KinesisSinkProvider();

@Override
public <T> SinkFunction<T> forSpec(EgressSpec<T> spec) {
final KinesisEgressSpec<T> kinesisEgressSpec = asKinesisEgressSpec(spec);
return delegateProvider.forSpec(kinesisEgressSpec);
}

private static <T> KinesisEgressSpec<T> asKinesisEgressSpec(EgressSpec<T> spec) {
if (!(spec instanceof JsonEgressSpec)) {
throw new IllegalArgumentException("Wrong type " + spec.type());
}
JsonEgressSpec<T> casted = (JsonEgressSpec<T>) spec;

EgressIdentifier<T> id = casted.id();
validateConsumedType(id);

JsonNode specJson = casted.specJson();

KinesisEgressBuilder<T> kinesisEgressBuilder = KinesisEgressBuilder.forIdentifier(id);

optionalAwsRegion(specJson).ifPresent(kinesisEgressBuilder::withAwsRegion);
optionalAwsCredentials(specJson).ifPresent(kinesisEgressBuilder::withAwsCredentials);
optionalMaxOutstandingRecords(specJson)
.ifPresent(kinesisEgressBuilder::withMaxOutstandingRecords);
KinesisIngressSpecJsonParser.clientConfigProperties(specJson)
.entrySet()
.forEach(
entry ->
kinesisEgressBuilder.withClientConfigurationProperty(
entry.getKey(), entry.getValue()));

kinesisEgressBuilder.withSerializer(serializerClass());

return kinesisEgressBuilder.build();
}

private static void validateConsumedType(EgressIdentifier<?> id) {
Class<?> consumedType = id.consumedType();
if (Any.class != consumedType) {
throw new IllegalArgumentException(
"Generic Kinesis egress is only able to consume messages types of "
+ Any.class.getName()
+ " but "
+ consumedType.getName()
+ " is provided.");
}
}

@SuppressWarnings("unchecked")
private static <T> Class<T> serializerClass() {
// this cast is safe, because we've already validated that the consumed type is Any.
return (Class<T>) GenericKinesisEgressSerializer.class;
}
}
Loading