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 2019/02/11 13:19:24 UTC

[GitHub] NicoK commented on a change in pull request #6705: [FLINK-10356][network] add sanity checks to SpillingAdaptiveSpanningRecordDeserializer

NicoK commented on a change in pull request #6705: [FLINK-10356][network] add sanity checks to SpillingAdaptiveSpanningRecordDeserializer
URL: https://github.com/apache/flink/pull/6705#discussion_r252367679
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
 ##########
 @@ -105,11 +119,235 @@ public void testHandleMixedLargeRecords() throws Exception {
 		testSerializationRoundTrip(originalRecords, segmentSize);
 	}
 
+	/**
+	 * Non-spanning, deserialization reads one byte too many and succeeds - failure report comes
+	 * from an additional check in {@link SpillingAdaptiveSpanningRecordDeserializer}.
+	 */
+	@Test
+	public void testHandleDeserializingTooMuchNonSpanning1() throws Exception {
+		testHandleWrongDeserialization(
+			DeserializingTooMuch.getValue(),
+			32 * 1024);
+	}
+
+	/**
+	 * Non-spanning, deserialization reads one byte too many and fails.
+	 */
+	@Test
+	public void testHandleDeserializingTooMuchNonSpanning2() throws Exception {
+		testHandleWrongDeserialization(
+			DeserializingTooMuch.getValue(),
+			(serializedLength) -> serializedLength,
+			isA(IndexOutOfBoundsException.class));
+	}
+
+	/**
+	 * Spanning, deserialization reads one byte too many and fails.
+	 */
+	@Test
+	public void testHandleDeserializingTooMuchSpanning1() throws Exception {
+		testHandleWrongDeserialization(
+			DeserializingTooMuch.getValue(),
+			(serializedLength) -> serializedLength - 1,
+			isA(EOFException.class));
+	}
+
+	/**
+	 * Spanning, deserialization reads one byte too many and fails.
+	 */
+	@Test
+	public void testHandleDeserializingTooMuchSpanning2() throws Exception {
+		testHandleWrongDeserialization(
+			DeserializingTooMuch.getValue(),
+			(serializedLength) -> 1,
+			isA(EOFException.class));
+	}
+
+	/**
+	 * Spanning, spilling, deserialization reads one byte too many.
+	 */
+	@Test
+	public void testHandleDeserializingTooMuchSpanningLargeRecord() throws Exception {
+		testHandleWrongDeserialization(
+			LargeObjectTypeDeserializingTooMuch.getRandom(),
+			32 * 1024,
+			isA(EOFException.class));
+	}
+
+	/**
+	 * Non-spanning, deserialization forgets to read one byte - failure report comes from an
+	 * additional check in {@link SpillingAdaptiveSpanningRecordDeserializer}.
+	 */
+	@Test
+	public void testHandleDeserializingNotEnoughNonSpanning() throws Exception {
+		testHandleWrongDeserialization(
+			DeserializingNotEnough.getValue(),
+			32 * 1024);
+	}
+
+	/**
+	 * Spanning, deserialization forgets to read one byte - failure report comes from an additional
+	 * check in {@link SpillingAdaptiveSpanningRecordDeserializer}.
+	 */
+	@Test
+	public void testHandleDeserializingNotEnoughSpanning1() throws Exception {
+		testHandleWrongDeserialization(
+			DeserializingNotEnough.getValue(),
+			(serializedLength) -> serializedLength - 1);
+	}
+
+	/**
+	 * Spanning, serialization length is 17 (including headers), deserialization forgets to read one
+	 * byte - failure report comes from an additional check in {@link SpillingAdaptiveSpanningRecordDeserializer}.
+	 */
+	@Test
+	public void testHandleDeserializingNotEnoughSpanning2() throws Exception {
+		testHandleWrongDeserialization(
+			DeserializingNotEnough.getValue(),
+			1);
+	}
+
+	/**
+	 * Spanning, spilling, deserialization forgets to read one byte - failure report comes from an
+	 * additional check in {@link SpillingAdaptiveSpanningRecordDeserializer}.
+	 */
+	@Test
+	public void testHandleDeserializingNotEnoughSpanningLargeRecord() throws Exception {
+		testHandleWrongDeserialization(
+			LargeObjectTypeDeserializingNotEnough.getRandom(),
+			32 * 1024);
+	}
+
+	private void testHandleWrongDeserialization(
+			WrongDeserializationValue testValue,
+			IntFunction<Integer> segmentSizeProvider,
+			Matcher<? extends Throwable> expectedCause) throws Exception {
+		expectedException.expectCause(expectedCause);
+		testHandleWrongDeserialization(testValue, segmentSizeProvider);
+	}
+
+	private void testHandleWrongDeserialization(
+			WrongDeserializationValue testValue,
+			@SuppressWarnings("SameParameterValue") int segmentSize,
+			Matcher<? extends Throwable> expectedCause) throws Exception {
+		expectedException.expectCause(expectedCause);
+		testHandleWrongDeserialization(testValue, segmentSize);
+	}
+
+	private void testHandleWrongDeserialization(
+			WrongDeserializationValue testValue,
+			IntFunction<Integer> segmentSizeProvider) throws Exception {
+		int serializedBytes = getSerializedBytes(testValue);
+		int segmentSize = segmentSizeProvider.apply(serializedBytes);
+		testHandleWrongDeserialization(testValue, segmentSize);
+	}
+
+	/**
+	 * Executes the de/serialization round trip test with a serializer that consumes more or less
+	 * bytes than what it writes and verifies the expected exception is thrown.
+	 */
+	private void testHandleWrongDeserialization(
+			WrongDeserializationValue testValue,
+			int segmentSize) throws Exception {
+		List<IOReadableWritable> originalRecords = new ArrayList<>(1);
+		originalRecords.add(testValue);
+
+		expectedException.expect(IOException.class);
+		if (testValue instanceof DeserializingTooMuch) {
+			expectedException.expectMessage(" -1 remaining unread byte");
+		} else if (testValue instanceof DeserializingNotEnough) {
+			expectedException.expectMessage(" 1 remaining unread byte");
+		} else {
+			fail("Invalid test value: " + testValue);
+		}
+
+		testSerializationRoundTrip(originalRecords, segmentSize);
+	}
+
+	/**
+	 * Retrieves the number of bytes the serialized representation of the given value takes (by
+	 * doing the serialization).
+	 */
+	private int getSerializedBytes(IOReadableWritable testValue) throws IOException {
+		SpanningRecordSerializer<IOReadableWritable> serializer = new SpanningRecordSerializer<>();
+		serializer.serializeRecord(testValue);
+		BufferBuilder bufferBuilder = createBufferBuilder();
+		assertTrue("Incorrect test setup: buffer not big enough to contain test value",
+			serializer.copyToBufferBuilder(bufferBuilder).isFullRecord());
+		int writtenBytes = bufferBuilder.finish();
+		bufferBuilder.createBufferConsumer().close();
+		return writtenBytes;
+	}
+
+	/**
+	 * Large object that tries to deserialize an additional byte that it has not written during
+	 * serialization.
+	 */
+	public static class LargeObjectTypeDeserializingTooMuch extends LargeObjectType implements DeserializingTooMuch {
+
+		@SuppressWarnings("WeakerAccess")
+		public LargeObjectTypeDeserializingTooMuch() {
+		}
+
+		@SuppressWarnings("WeakerAccess")
+		public LargeObjectTypeDeserializingTooMuch(int len) {
+			super(len);
+		}
+
+		@Override
+		public void read(DataInputView in) throws IOException {
+			super.read(in);
+			in.readUnsignedByte(); // not written by write()
+		}
+
+		@Override
+		public LargeObjectTypeDeserializingTooMuch getRandom(Random rnd) {
+			int length = super.getRandom(rnd).length();
+			return new LargeObjectTypeDeserializingTooMuch(length);
+		}
+
+		static LargeObjectTypeDeserializingTooMuch getRandom() {
+			return (new LargeObjectTypeDeserializingTooMuch()).getRandom(new Random());
 
 Review comment:
   Using a fixed seed is not preferable: you'd always test the same record(s) while otherwise each test would test something else and you get better test coverage over time. (I don't know why `org.apache.flink.testutils.serialization.types.Util` is using that pattern though)
   
   I'll have a quick look on whether it is possible to print something generic (the seed) in case of unit test failure - the latter part can be done manually when trying to fix an error.

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