@Override public KryoSerializer<T> duplicate() { return new KryoSerializer<>(this); }
private void runDuplicateSerializerTest(ExecutionConfig executionConfig) { final KryoSerializer<WrappedString> original = new KryoSerializer<>(WrappedString.class, executionConfig); final KryoSerializer<WrappedString> duplicate = original.duplicate(); WrappedString testString = new WrappedString("test"); String copyWithOriginal = original.copy(testString).content; String copyWithDuplicate = duplicate.copy(testString).content; Assert.assertTrue(copyWithOriginal.startsWith(testString.content)); Assert.assertTrue(copyWithDuplicate.startsWith(testString.content)); // check that both serializer instances have appended a different identity hash Assert.assertNotEquals(copyWithOriginal, copyWithDuplicate); }
@Override public void copy(DataInputView source, DataOutputView target) throws IOException { if (CONCURRENT_ACCESS_CHECK) { enterExclusiveThread(); } try { checkKryoInitialized(); if (this.copyInstance == null){ this.copyInstance = createInstance(); } T tmp = deserialize(copyInstance, source); serialize(tmp, target); } finally { if (CONCURRENT_ACCESS_CHECK) { exitExclusiveThread(); } } }
@Test public void validateReferenceMappingEnabled() { KryoSerializer<String> serializer = new KryoSerializer<>(String.class, new ExecutionConfig()); Kryo kryo = serializer.getKryo(); assertTrue(kryo.getReferences()); } }
KryoSerializer<Integer> serializer = new KryoSerializer<>(Integer.class, new ExecutionConfig()); serializer.serialize(i, target); int value = serializer.deserialize(source); assertEquals(i, value); serializer.deserialize(source); fail("should throw a java.io.EOFException");
KryoSerializer<String> serializer = new KryoSerializer<String>(String.class, new ExecutionConfig()); serializer.serialize(str, target); fail("should throw a java.io.EOFException");
new KryoSerializer<>(FakeClass.class, executionConfig); DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(f)) { FakeClass myTestClass = kryoSerializer.deserialize(inputView);
executionConfig.registerKryoType(TestClassB.class); KryoSerializer<TestClass> kryoSerializer = new KryoSerializer<>(TestClass.class, executionConfig); int testClassId = kryoSerializer.getKryo().getRegistration(TestClass.class).getId(); int testClassAId = kryoSerializer.getKryo().getRegistration(TestClassA.class).getId(); int testClassBId = kryoSerializer.getKryo().getRegistration(TestClassB.class).getId(); TypeSerializerSnapshot kryoSerializerConfigSnapshot = kryoSerializer.snapshotConfiguration(); byte[] serializedConfig; try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { executionConfig.registerKryoType(TestClassA.class); kryoSerializer = new KryoSerializer<>(TestClass.class, executionConfig); kryoSerializerConfigSnapshot.resolveSchemaCompatibility(kryoSerializer); assertTrue(compatResult.isCompatibleAsIs()); assertEquals(testClassId, kryoSerializer.getKryo().getRegistration(TestClass.class).getId()); assertEquals(testClassAId, kryoSerializer.getKryo().getRegistration(TestClassA.class).getId()); assertEquals(testClassBId, kryoSerializer.getKryo().getRegistration(TestClassB.class).getId());
/** * Verifies that reconfiguration result is INCOMPATIBLE if data type has changed. */ @Test public void testMigrationStrategyWithDifferentKryoType() throws Exception { KryoSerializer<TestClassA> kryoSerializerForA = new KryoSerializer<>(TestClassA.class, new ExecutionConfig()); // snapshot configuration and serialize to bytes TypeSerializerSnapshot kryoSerializerConfigSnapshot = kryoSerializerForA.snapshotConfiguration(); byte[] serializedConfig; try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { TypeSerializerSnapshotSerializationUtil.writeSerializerSnapshot( new DataOutputViewStreamWrapper(out), kryoSerializerConfigSnapshot, kryoSerializerForA); serializedConfig = out.toByteArray(); } KryoSerializer<TestClassB> kryoSerializerForB = new KryoSerializer<>(TestClassB.class, new ExecutionConfig()); // read configuration again from bytes try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { kryoSerializerConfigSnapshot = TypeSerializerSnapshotSerializationUtil.readSerializerSnapshot( new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader(), kryoSerializerForB); } @SuppressWarnings("unchecked") TypeSerializerSchemaCompatibility<TestClassB> compatResult = kryoSerializerConfigSnapshot.resolveSchemaCompatibility(kryoSerializerForB); assertTrue(compatResult.isIncompatible()); }
@Test public void testReducingStateInstantiation() throws Exception { final ExecutionConfig config = new ExecutionConfig(); config.registerKryoType(Path.class); final AtomicReference<Object> descriptorCapture = new AtomicReference<>(); StreamingRuntimeContext context = new StreamingRuntimeContext( createDescriptorCapturingMockOp(descriptorCapture, config), createMockEnvironment(), Collections.<String, Accumulator<?, ?>>emptyMap()); @SuppressWarnings("unchecked") ReduceFunction<TaskInfo> reducer = (ReduceFunction<TaskInfo>) mock(ReduceFunction.class); ReducingStateDescriptor<TaskInfo> descr = new ReducingStateDescriptor<>("name", reducer, TaskInfo.class); context.getReducingState(descr); StateDescriptor<?, ?> descrIntercepted = (StateDescriptor<?, ?>) descriptorCapture.get(); TypeSerializer<?> serializer = descrIntercepted.getSerializer(); // check that the Path class is really registered, i.e., the execution config was applied assertTrue(serializer instanceof KryoSerializer); assertTrue(((KryoSerializer<?>) serializer).getKryo().getRegistration(Path.class).getId() > 0); }
@Override public T deserialize(T reuse, DataInputView source) throws IOException { return deserialize(source); }
@Override public T copy(T from, T reuse) { return copy(from); }
private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); // kryoRegistrations may be null if this Kryo serializer is deserialized from an old version if (kryoRegistrations == null) { this.kryoRegistrations = buildKryoRegistrations( type, registeredTypes, registeredTypesWithSerializerClasses, registeredTypesWithSerializers); } }
@Override public void go() throws Exception { serializer.serialize("a value", lockingOut); } };
/** * Tests that the registered classes in Kryo did not change. * * <p>Once we have proper serializer versioning this test will become obsolete. * But currently a change in the serializers can break savepoint backwards * compatibility between Flink versions. */ @Test public void testDefaultKryoRegisteredClassesDidNotChange() throws Exception { final Kryo kryo = new KryoSerializer<>(Integer.class, new ExecutionConfig()).getKryo(); try (BufferedReader reader = new BufferedReader(new InputStreamReader( getClass().getClassLoader().getResourceAsStream("flink_11-kryo_registrations")))) { String line; while ((line = reader.readLine()) != null) { String[] split = line.split(","); final int tag = Integer.parseInt(split[0]); final String registeredClass = split[1]; Registration registration = kryo.getRegistration(tag); if (registration == null) { fail(String.format("Registration for %d = %s got lost", tag, registeredClass)); } else if (!registeredClass.equals(registration.getType().getName())) { fail(String.format("Registration for %d = %s changed to %s", tag, registeredClass, registration.getType().getName())); } } } }
executionConfig.registerKryoType(TestRecord.class); KryoSerializer<TestRecord> kryoSerializer = new KryoSerializer<TestRecord>( TestRecord.class, executionConfig); kryoSerializer.serialize(testRecord, target); kryoSerializer.serialize(testRecord, target); Assert.fail("Expected an EOFException."); } catch(EOFException eofException) { TestRecord actualRecord = kryoSerializer.deserialize( new DataInputViewStreamWrapper(new ByteArrayInputStream(target.getBuffer()))); kryoSerializer.serialize(testRecord, target);
@Test public void testConcurrentUseOfSerializer() throws Exception { final KryoSerializer<String> serializer = new KryoSerializer<>(String.class, new ExecutionConfig()); final BlockerSync sync = new BlockerSync(); final DataOutputView regularOut = new DataOutputSerializer(32); final DataOutputView lockingOut = new LockingView(sync); // this thread serializes and gets stuck there final CheckedThread thread = new CheckedThread("serializer") { @Override public void go() throws Exception { serializer.serialize("a value", lockingOut); } }; thread.start(); sync.awaitBlocker(); // this should fail with an exception try { serializer.serialize("value", regularOut); fail("should have failed with an exception"); } catch (IllegalStateException e) { // expected } finally { // release the thread that serializes sync.releaseBlocker(); } // this propagates exceptions from the spawned thread thread.sync(); }
new KryoSerializer<>(FakeAvroClass.class, executionConfig); kryoSerializer.deserialize(inputView);
@Test public void testAggregatingStateInstantiation() throws Exception { final ExecutionConfig config = new ExecutionConfig(); config.registerKryoType(Path.class); final AtomicReference<Object> descriptorCapture = new AtomicReference<>(); StreamingRuntimeContext context = new StreamingRuntimeContext( createDescriptorCapturingMockOp(descriptorCapture, config), createMockEnvironment(), Collections.<String, Accumulator<?, ?>>emptyMap()); @SuppressWarnings("unchecked") AggregateFunction<String, TaskInfo, String> aggregate = (AggregateFunction<String, TaskInfo, String>) mock(AggregateFunction.class); AggregatingStateDescriptor<String, TaskInfo, String> descr = new AggregatingStateDescriptor<>("name", aggregate, TaskInfo.class); context.getAggregatingState(descr); AggregatingStateDescriptor<?, ?, ?> descrIntercepted = (AggregatingStateDescriptor<?, ?, ?>) descriptorCapture.get(); TypeSerializer<?> serializer = descrIntercepted.getSerializer(); // check that the Path class is really registered, i.e., the execution config was applied assertTrue(serializer instanceof KryoSerializer); assertTrue(((KryoSerializer<?>) serializer).getKryo().getRegistration(Path.class).getId() > 0); }
@Override public T deserialize(T reuse, DataInputView source) throws IOException { return deserialize(source); }