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 2018/10/18 08:34:17 UTC

[GitHub] asfgit closed pull request #6860: [FLINK-10567]Fix TtlStateSerializer lost field during duplicate()

asfgit closed pull request #6860: [FLINK-10567]Fix TtlStateSerializer lost field during duplicate()
URL: https://github.com/apache/flink/pull/6860
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlStateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlStateFactory.java
index 2f291d3693b..45f4e3bf143 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlStateFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlStateFactory.java
@@ -224,7 +224,7 @@ protected Object getField(@Nonnull TtlValue<T> v, int index) {
 			TypeSerializer<?> ... originalSerializers) {
 			Preconditions.checkNotNull(originalSerializers);
 			Preconditions.checkArgument(originalSerializers.length == 2);
-			return new TtlSerializer<>(precomputed, (TypeSerializer<T>) originalSerializers[1]);
+			return new TtlSerializer<>(precomputed, originalSerializers);
 		}
 	}
 }
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
index 649c6d03a6a..0b26129fcf3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
@@ -33,8 +33,10 @@
 import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.StateTtlConfig;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.ParameterlessTypeSerializerConfig;
@@ -1340,6 +1342,23 @@ public void testValueState() throws Exception {
 		backend.dispose();
 	}
 
+	@Test
+	public void testValueStateWorkWithTtl() throws Exception {
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+		try {
+			ValueStateDescriptor<MutableLong> kvId = new ValueStateDescriptor<>("id", MutableLong.class);
+			kvId.enableTimeToLive(StateTtlConfig.newBuilder(Time.seconds(1)).build());
+
+			ValueState<MutableLong> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+			backend.setCurrentKey(1);
+			state.update(new MutableLong());
+			state.value();
+		} finally {
+			backend.close();
+			backend.dispose();
+		}
+	}
+
 	/**
 	 * Tests {@link ValueState#value()} and
 	 * {@link InternalKvState#getSerializedValue(byte[], TypeSerializer, TypeSerializer, TypeSerializer)}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


With regards,
Apache Git Services