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/08/18 08:55:58 UTC

[GitHub] [flink] xiaolong-sn commented on a change in pull request #13102: [FLINK-18515][Kinesis] Adding FanOutRecordPublisher for Kinesis EFO support

xiaolong-sn commented on a change in pull request #13102:
URL: https://github.com/apache/flink/pull/13102#discussion_r470397615



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
##########
@@ -389,25 +424,54 @@ private RecordEmitter createRecordEmitter(Properties configProps) {
 	 * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
 	 * @param subscribedShard the shard this consumer is subscribed to
 	 * @param lastSequenceNum the sequence number in the shard to start consuming
-	 * @param shardMetricsReporter the reporter to report metrics to
+	 * @param metricGroup the metric group to report metrics to
 	 * @return shard consumer
 	 */
 	protected ShardConsumer<T> createShardConsumer(
-		Integer subscribedShardStateIndex,
-		StreamShardHandle subscribedShard,
-		SequenceNumber lastSequenceNum,
-		ShardMetricsReporter shardMetricsReporter,
-		KinesisDeserializationSchema<T> shardDeserializer) {
+			final Integer subscribedShardStateIndex,
+			final StreamShardHandle subscribedShard,
+			final SequenceNumber lastSequenceNum,
+			final MetricGroup metricGroup,
+			final KinesisDeserializationSchema<T> shardDeserializer) throws InterruptedException {
+
 		return new ShardConsumer<>(
 			this,
+			createRecordPublisher(lastSequenceNum, configProps, metricGroup, subscribedShard),
 			subscribedShardStateIndex,
 			subscribedShard,
 			lastSequenceNum,
-			this.kinesisProxyFactory.create(configProps),
-			shardMetricsReporter,
+			new ShardConsumerMetricsReporter(metricGroup),
 			shardDeserializer);
 	}
 
+	protected RecordPublisherFactory createRecordPublisherFactory() {
+		RecordPublisherType recordPublisherType = RecordPublisherType.valueOf(
+			configProps.getProperty(RECORD_PUBLISHER_TYPE, POLLING.name()));
+
+		switch (recordPublisherType) {
+			case EFO:
+				return new FanOutRecordPublisherFactory(kinesisProxyV2Factory.create(configProps));
+			case POLLING:

Review comment:
       The switch term may cause a misunderstanding when the recordPublisherType is POLLING.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
##########
@@ -330,23 +359,26 @@ public KinesisDataFetcher(List<String> streams,
 			new AtomicReference<>(),
 			new ArrayList<>(),
 			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
-			KinesisProxy::create);
+			KinesisProxy::create,
+			KinesisDataFetcher::createKinesisProxyV2);
 	}
 
 	@VisibleForTesting
-	protected KinesisDataFetcher(List<String> streams,
-								SourceFunction.SourceContext<T> sourceContext,
-								Object checkpointLock,
-								RuntimeContext runtimeContext,
-								Properties configProps,
-								KinesisDeserializationSchema<T> deserializationSchema,
-								KinesisShardAssigner shardAssigner,
-								AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner,
-								WatermarkTracker watermarkTracker,
-								AtomicReference<Throwable> error,
-								List<KinesisStreamShardState> subscribedShardsState,
-								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
-								FlinkKinesisProxyFactory kinesisProxyFactory) {
+	protected KinesisDataFetcher(
+			final List<String> streams,
+			final SourceFunction.SourceContext<T> sourceContext,
+			final Object checkpointLock,
+			final RuntimeContext runtimeContext,
+			final Properties configProps,
+			final KinesisDeserializationSchema<T> deserializationSchema,
+			final KinesisShardAssigner shardAssigner,
+			final AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner,
+			final WatermarkTracker watermarkTracker,
+			final AtomicReference<Throwable> error,
+			final List<KinesisStreamShardState> subscribedShardsState,
+			final HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
+			final FlinkKinesisProxyFactory kinesisProxyFactory,
+			@Nullable final FlinkKinesisProxyV2Factory kinesisProxyV2Factory) {

Review comment:
       The reformat is excellent, it cures my OCD.

##########
File path: flink-connectors/flink-connector-kinesis/pom.xml
##########
@@ -161,6 +163,30 @@ under the License.
 			<scope>provided</scope>
 		</dependency>
 
+		<!-- Amazon AWS SDK v2.x dependencies -->
+		<dependency>
+			<groupId>software.amazon.awssdk</groupId>
+			<artifactId>kinesis</artifactId>
+			<version>${aws.sdkv2.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>software.amazon.awssdk</groupId>
+			<artifactId>netty-nio-client</artifactId>
+			<version>${aws.sdkv2.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>software.amazon.awssdk</groupId>
+			<artifactId>sts</artifactId>
+			<version>${aws.sdkv2.version}</version>
+		</dependency>
+
+		<dependency>

Review comment:
       This dependency may lead to an exception when trying to consume from enhanced fan-out. The detailed info has been informed.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisher.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.internals.publisher.fanout;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.fanout.FanOutShardSubscriber.FanOutSubscriberException;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.FullJitterBackoff;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2Interface;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.EncryptionType;
+import software.amazon.awssdk.services.kinesis.model.Record;
+import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEvent;
+
+import javax.annotation.Nonnull;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.function.Consumer;
+
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+import static software.amazon.awssdk.services.kinesis.model.StartingPosition.builder;
+
+/**
+ * A {@link RecordPublisher} that will read and forward records from Kinesis using EFO, to the subscriber.
+ * Records are consumed via Enhanced Fan Out subscriptions using SubscribeToShard API.
+ */
+@Internal
+public class FanOutRecordPublisher implements RecordPublisher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(FanOutRecordPublisher.class);
+
+	private final FullJitterBackoff backoff;
+
+	private final String consumerArn;
+
+	private final KinesisProxyV2Interface kinesisProxy;
+
+	private final StreamShardHandle subscribedShard;
+
+	private final FanOutRecordPublisherConfiguration configuration;
+
+	/** The current attempt in the case of subsequent recoverable errors. */
+	private int attempt = 0;
+
+	private StartingPosition nextStartingPosition;
+
+	/**
+	 * Instantiate a new FanOutRecordPublisher.
+	 * Consumes data from KDS using EFO SubscribeToShard over AWS SDK V2.x
+	 *
+	 * @param startingPosition the position in the shard to start consuming from
+	 * @param consumerArn the consumer ARN of the stream consumer
+	 * @param subscribedShard the shard to consumer from
+	 * @param kinesisProxy the proxy used to talk to Kinesis services
+	 * @param configuration the record publisher configuration
+	 */
+	public FanOutRecordPublisher(
+			final StartingPosition startingPosition,
+			final String consumerArn,
+			final StreamShardHandle subscribedShard,
+			final KinesisProxyV2Interface kinesisProxy,
+			final FanOutRecordPublisherConfiguration configuration,
+			final FullJitterBackoff backoff) {
+		this.nextStartingPosition = Preconditions.checkNotNull(startingPosition);
+		this.consumerArn = Preconditions.checkNotNull(consumerArn);
+		this.subscribedShard = Preconditions.checkNotNull(subscribedShard);
+		this.kinesisProxy = Preconditions.checkNotNull(kinesisProxy);
+		this.configuration = Preconditions.checkNotNull(configuration);
+		this.backoff = Preconditions.checkNotNull(backoff);
+	}
+
+	@Override
+	public RecordPublisherRunResult run(final RecordBatchConsumer recordConsumer) throws InterruptedException {
+		LOG.info("Running fan out record publisher on {}::{} from {} - {}",
+			subscribedShard.getStreamName(),
+			subscribedShard.getShard().getShardId(),
+			nextStartingPosition.getShardIteratorType(),
+			nextStartingPosition.getStartingMarker());
+
+		Consumer<SubscribeToShardEvent> eventConsumer = event -> {
+			RecordBatch recordBatch = new RecordBatch(toSdkV1Records(event.records()), subscribedShard, event.millisBehindLatest());

Review comment:
       I'm curious about the overhead workload of the function `toSdkV1Records`.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisherFactory.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.internals.publisher.fanout;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisherFactory;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.FullJitterBackoff;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2Interface;
+import org.apache.flink.util.Preconditions;
+
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+
+import java.util.Optional;
+import java.util.Properties;
+
+import static java.util.Collections.singletonList;
+
+/**
+ * A {@link RecordPublisher} factory used to create instances of {@link FanOutRecordPublisher}.
+ */
+@Internal
+public class FanOutRecordPublisherFactory implements RecordPublisherFactory {
+
+	private static final FullJitterBackoff BACKOFF = new FullJitterBackoff();
+
+	/**
+	 * A singleton {@link KinesisProxyV2} is used per Flink task.
+	 * The {@link KinesisAsyncClient} uses an internal thread pool; using a single client reduces overhead.
+	 */
+	private final KinesisProxyV2Interface kinesisProxy;
+
+	/**
+	 * Instantiate a factory responsible for creating {@link FanOutRecordPublisher}.
+	 *
+	 * @param kinesisProxy the singleton proxy used by all record publishers created by this factory
+	 */
+	public FanOutRecordPublisherFactory(final KinesisProxyV2Interface kinesisProxy) {
+		this.kinesisProxy = kinesisProxy;
+	}
+
+	/**
+	 * Create a {@link FanOutRecordPublisher}.
+	 *
+	 * @param startingPosition the starting position in the shard to start consuming from
+	 * @param consumerConfig the consumer configuration properties
+	 * @param metricGroup the metric group to report metrics to
+	 * @param streamShardHandle the shard this consumer is subscribed to
+	 * @return a {@link FanOutRecordPublisher}
+	 */
+	@Override
+	public FanOutRecordPublisher create(
+			final StartingPosition startingPosition,
+			final Properties consumerConfig,
+			final MetricGroup metricGroup,
+			final StreamShardHandle streamShardHandle) {
+		Preconditions.checkNotNull(startingPosition);
+		Preconditions.checkNotNull(consumerConfig);
+		Preconditions.checkNotNull(metricGroup);
+		Preconditions.checkNotNull(streamShardHandle);
+
+		String stream = streamShardHandle.getStreamName();
+		FanOutRecordPublisherConfiguration configuration = new FanOutRecordPublisherConfiguration(consumerConfig, singletonList(stream));
+
+		Optional<String> streamConsumerArn = configuration.getStreamConsumerArn(stream);

Review comment:
       This streamConsumerArn part is not done, right?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StartingPosition.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.model;
+
+import org.apache.flink.annotation.Internal;
+
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+
+import javax.annotation.Nullable;
+
+import java.util.Date;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.TRIM_HORIZON;
+import static org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber.isSentinelSequenceNumber;
+
+/**
+ * The position in which to start consuming from a stream.
+ */
+@Internal
+public class StartingPosition {
+
+	private final ShardIteratorType shardIteratorType;
+
+	private final Object startingMarker;
+
+	private StartingPosition(final ShardIteratorType shardIteratorType, @Nullable final Object startingMarker) {
+		this.shardIteratorType = shardIteratorType;
+		this.startingMarker = startingMarker;
+	}
+
+	public ShardIteratorType getShardIteratorType() {
+		return shardIteratorType;
+	}
+
+	@Nullable
+	public Object getStartingMarker() {
+		return startingMarker;
+	}
+
+	public static StartingPosition fromTimestamp(final Date date) {

Review comment:
       The java doc is missing here. 




----------------------------------------------------------------
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