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-24943][Connectors / Kinesis] Explicitly create KryoSerializer for SequenceNumber class in Kinesis Consumer #20257

Open
wants to merge 3 commits into
base: master
Choose a base branch
from
Open
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 @@ -28,7 +28,6 @@
import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
Expand All @@ -46,6 +45,7 @@
import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper;
import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
import org.apache.flink.streaming.connectors.kinesis.util.KinesisStateUtil;
import org.apache.flink.streaming.connectors.kinesis.util.StreamConsumerRegistrarUtil;
import org.apache.flink.streaming.connectors.kinesis.util.WatermarkTracker;
import org.apache.flink.util.InstantiationUtil;
Expand Down Expand Up @@ -228,7 +228,8 @@ public FlinkKinesisConsumer(
"The provided deserialization schema is not serializable: "
+ deserializer.getClass().getName()
+ ". "
+ "Please check that it does not contain references to non-serializable instances.");
+ "Please check that it does not contain references "
+ "to non-serializable instances.");
this.deserializer = deserializer;

StreamConsumerRegistrarUtil.eagerlyRegisterStreamConsumers(configProps, streams);
Expand Down Expand Up @@ -370,7 +371,8 @@ public void run(SourceContext<T> sourceContext) throws Exception {

if (LOG.isInfoEnabled()) {
LOG.info(
"Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}",
"Subtask {} will be seeded with initial shard {}, "
+ "starting state set as sequence number {}",
getRuntimeContext().getIndexOfThisSubtask(),
shard.toString(),
startingSeqNum.get());
Expand Down Expand Up @@ -434,16 +436,13 @@ public TypeInformation<T> getProducedType() {

@Override
public void initializeState(FunctionInitializationContext context) throws Exception {
TypeInformation<Tuple2<StreamShardMetadata, SequenceNumber>> shardsStateTypeInfo =
new TupleTypeInfo<>(
TypeInformation.of(StreamShardMetadata.class),
TypeInformation.of(SequenceNumber.class));

sequenceNumsStateForCheckpoint =
context.getOperatorStateStore()
.getUnionListState(
new ListStateDescriptor<>(
sequenceNumsStateStoreName, shardsStateTypeInfo));
sequenceNumsStateStoreName,
KinesisStateUtil.createShardsStateSerializer(
getRuntimeContext().getExecutionConfig())));

if (context.isRestored()) {
if (sequenceNumsToRestore == null) {
Expand All @@ -463,7 +462,8 @@ public void initializeState(FunctionInitializationContext context) throws Except
}

LOG.info(
"Setting restore state in the FlinkKinesisConsumer. Using the following offsets: {}",
"Setting restore state in the FlinkKinesisConsumer. "
+ "Using the following offsets: {}",
sequenceNumsToRestore);
}
} else {
Expand Down Expand Up @@ -509,7 +509,8 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception {

if (LOG.isDebugEnabled()) {
LOG.debug(
"Snapshotted state, last processed sequence numbers: {}, checkpoint id: {}, timestamp: {}",
"Snapshotted state, last processed sequence numbers: {}, "
+ "checkpoint id: {}, timestamp: {}",
lastStateSnapshot,
context.getCheckpointId(),
context.getCheckpointTimestamp());
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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.util;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
import org.apache.flink.streaming.connectors.kinesis.model.StreamShardMetadata;

/** Utilities for Flink Kinesis connector state management. */
public class KinesisStateUtil {

/** To prevent instantiation of class. */
private KinesisStateUtil() {}

/**
* Creates state serializer for kinesis shard sequence number. Using of the explicit state
* serializer with KryoSerializer is needed because otherwise users cannot use
* 'disableGenericTypes' properties with KinesisConsumer, see FLINK-24943 for details
*
* @return state serializer
*/
public static TupleSerializer<Tuple2<StreamShardMetadata, SequenceNumber>>
createShardsStateSerializer(ExecutionConfig executionConfig) {
// explicit serializer will keep the compatibility with GenericTypeInformation
// and allow to disableGenericTypes for users
TypeSerializer<?>[] fieldSerializers =
new TypeSerializer<?>[] {
TypeInformation.of(StreamShardMetadata.class).createSerializer(executionConfig),
new KryoSerializer<>(SequenceNumber.class, executionConfig)
};
@SuppressWarnings("unchecked")
Class<Tuple2<StreamShardMetadata, SequenceNumber>> tupleClass =
(Class<Tuple2<StreamShardMetadata, SequenceNumber>>) (Class<?>) Tuple2.class;
return new TupleSerializer<>(tupleClass, fieldSerializers);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,11 @@
import org.apache.flink.api.common.state.OperatorStateStore;
import org.apache.flink.api.common.time.Deadline;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.runtime.PojoSerializer;
import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.mock.Whitebox;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
Expand All @@ -54,10 +57,13 @@
import org.apache.flink.streaming.connectors.kinesis.testutils.TestUtils;
import org.apache.flink.streaming.connectors.kinesis.testutils.TestableFlinkKinesisConsumer;
import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
import org.apache.flink.streaming.connectors.kinesis.util.KinesisStateUtil;
import org.apache.flink.streaming.connectors.kinesis.util.RecordEmitter;
import org.apache.flink.streaming.connectors.kinesis.util.WatermarkTracker;
import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
import org.apache.flink.streaming.util.CollectingSourceContext;
import org.apache.flink.streaming.util.MockStreamingRuntimeContext;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.TestLogger;

import com.amazonaws.services.kinesis.model.HashKeyRange;
Expand Down Expand Up @@ -160,9 +166,7 @@ public void testUseRestoredStateForSnapshotIfFetcherNotInitialized() throws Exce

FlinkKinesisConsumer<String> consumer =
new FlinkKinesisConsumer<>("fakeStream", new SimpleStringSchema(), config);
RuntimeContext context = mock(RuntimeContext.class);
when(context.getIndexOfThisSubtask()).thenReturn(0);
when(context.getNumberOfParallelSubtasks()).thenReturn(2);
RuntimeContext context = new MockStreamingRuntimeContext(true, 2, 0);
consumer.setRuntimeContext(context);

OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
Expand Down Expand Up @@ -280,8 +284,7 @@ public void testListStateChangedAfterSnapshotState() throws Exception {
new FlinkKinesisConsumer<>("fakeStream", new SimpleStringSchema(), config);
FlinkKinesisConsumer<?> mockedConsumer = spy(consumer);

RuntimeContext context = mock(RuntimeContext.class);
when(context.getIndexOfThisSubtask()).thenReturn(1);
RuntimeContext context = new MockStreamingRuntimeContext(true, 1, 1);

mockedConsumer.setRuntimeContext(context);
mockedConsumer.initializeState(initializationContext);
Expand Down Expand Up @@ -309,6 +312,52 @@ public void testListStateChangedAfterSnapshotState() throws Exception {
}
}

/**
* Before using an explicit TypeSerializer for the state the {@link FlinkKinesisConsumer} was
* creating a serializer implicitly using a {@link TypeInformation}. After fixing issue
* FLINK-24943, * serializer is created explicitly. Here, we verify that previous approach is
* compatible with the new one.
*/
@Test
public void testExplicitStateSerializerCompatibility() throws Exception {
ExecutionConfig executionConfig = new ExecutionConfig();

Tuple2<StreamShardMetadata, SequenceNumber> tuple =
new Tuple2<>(
KinesisDataFetcher.convertToStreamShardMetadata(
new StreamShardHandle(
"fakeStream",
new Shard()
.withShardId(
KinesisShardIdGenerator
.generateFromShardOrder(0)))),
new SequenceNumber("1"));

// This is how serializer was created implicitly using a TypeInformation
// and since SequenceNumber is GenericType, Flink falls back to Kryo
TypeInformation<Tuple2<StreamShardMetadata, SequenceNumber>> originalShardsStateTypeInfo =
new TupleTypeInfo<>(
TypeInformation.of(StreamShardMetadata.class),
TypeInformation.of(SequenceNumber.class));
TypeSerializer<Tuple2<StreamShardMetadata, SequenceNumber>> serializerFromTypeInfo =
originalShardsStateTypeInfo.createSerializer(executionConfig);
byte[] bytes = InstantiationUtil.serializeToByteArray(serializerFromTypeInfo, tuple);

// This is how we create serializer explicitly with Kryo
TupleSerializer<Tuple2<StreamShardMetadata, SequenceNumber>> serializerFromKryo =
KinesisStateUtil.createShardsStateSerializer(executionConfig);

Tuple2<StreamShardMetadata, SequenceNumber> actualTuple =
InstantiationUtil.deserializeFromByteArray(serializerFromKryo, bytes);

// Both ways should be the same
assertThat(tuple)
.overridingErrorMessage(
"Explicit serializer is not compatible with "
+ "implicit method of creating serializer using TypeInformation.")
.isEqualTo(actualTuple);
}

// ----------------------------------------------------------------------
// Tests related to fetcher initialization
// ----------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,7 @@
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.common.serialization.SimpleStringSchema;
import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;

import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.apache.flink.streaming.util.MockStreamingRuntimeContext;

import java.util.Properties;

Expand All @@ -39,27 +36,9 @@ public TestableFlinkKinesisConsumer(
final int indexOfThisConsumerSubtask) {
super(fakeStream, new SimpleStringSchema(), fakeConfiguration);

this.mockedRuntimeCtx = Mockito.mock(RuntimeContext.class);

Mockito.when(mockedRuntimeCtx.getNumberOfParallelSubtasks())
.thenAnswer(
new Answer<Integer>() {
@Override
public Integer answer(InvocationOnMock invocationOnMock)
throws Throwable {
return totalNumOfConsumerSubtasks;
}
});

Mockito.when(mockedRuntimeCtx.getIndexOfThisSubtask())
.thenAnswer(
new Answer<Integer>() {
@Override
public Integer answer(InvocationOnMock invocationOnMock)
throws Throwable {
return indexOfThisConsumerSubtask;
}
});
this.mockedRuntimeCtx =
new MockStreamingRuntimeContext(
true, totalNumOfConsumerSubtasks, indexOfThisConsumerSubtask);
}

@Override
Expand Down