[ 
https://issues.apache.org/jira/browse/FLINK-10356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16660296#comment-16660296
 ] 

ASF GitHub Bot commented on FLINK-10356:
----------------------------------------

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

 ##########
 File path: 
flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
 ##########
 @@ -62,127 +68,103 @@ public static Configuration getConfiguration() {
        }
 
        @Test
-       public void testIncorrectSerializer1() {
-               try {
-                       ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
-                       env.setParallelism(PARLLELISM);
-                       env.getConfig().disableSysoutLogging();
+       public void testIncorrectSerializer1() throws Exception {
+               expectedException.expect(JobExecutionException.class);
+               expectedException.expectCause(
+                       Matchers.both(isA(IOException.class))
+                               .and(hasProperty("message", 
containsString("broken serialization"))));
+
+               ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
+               env.setParallelism(PARLLELISM);
+               env.getConfig().disableSysoutLogging();
+
+               env
+                       .generateSequence(1, 10 * PARLLELISM)
+                       .map(new MapFunction<Long, ConsumesTooMuch>() {
+                               @Override
+                               public ConsumesTooMuch map(Long value) throws 
Exception {
+                                       return new ConsumesTooMuch();
+                               }
+                       })
+                       .rebalance()
+                       .output(new DiscardingOutputFormat<ConsumesTooMuch>());
+
+               env.execute();
+       }
+
+       @Test
+       public void testIncorrectSerializer2() throws Exception {
+               expectedException.expect(JobExecutionException.class);
+               expectedException.expectCause(
+                       Matchers.both(isA(IOException.class))
+                               .and(hasProperty("message", 
containsString("broken serialization"))));
+
+               ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
+               env.setParallelism(PARLLELISM);
+               env.getConfig().disableSysoutLogging();
 
-                       env
+               env
                                .generateSequence(1, 10 * PARLLELISM)
-                               .map(new MapFunction<Long, ConsumesTooMuch>() {
+                               .map(new MapFunction<Long, 
ConsumesTooMuchSpanning>() {
                                        @Override
-                                       public ConsumesTooMuch map(Long value) 
throws Exception {
-                                               return new ConsumesTooMuch();
+                                       public ConsumesTooMuchSpanning map(Long 
value) throws Exception {
+                                               return new 
ConsumesTooMuchSpanning();
                                        }
                                })
                                .rebalance()
-                               .output(new 
DiscardingOutputFormat<ConsumesTooMuch>());
+                               .output(new 
DiscardingOutputFormat<ConsumesTooMuchSpanning>());
 
-                       env.execute();
-               }
-               catch (JobExecutionException e) {
-                       Throwable rootCause = e.getCause();
-                       assertTrue(rootCause instanceof IOException);
-                       assertTrue(rootCause.getMessage().contains("broken 
serialization"));
-               }
-               catch (Exception e) {
-                       e.printStackTrace();
-                       fail(e.getMessage());
-               }
+               env.execute();
        }
 
        @Test
-       public void testIncorrectSerializer2() {
-               try {
-                       ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
-                       env.setParallelism(PARLLELISM);
-                       env.getConfig().disableSysoutLogging();
-
-                       env
-                                       .generateSequence(1, 10 * PARLLELISM)
-                                       .map(new MapFunction<Long, 
ConsumesTooMuchSpanning>() {
-                                               @Override
-                                               public ConsumesTooMuchSpanning 
map(Long value) throws Exception {
-                                                       return new 
ConsumesTooMuchSpanning();
-                                               }
-                                       })
-                                       .rebalance()
-                                       .output(new 
DiscardingOutputFormat<ConsumesTooMuchSpanning>());
-
-                       env.execute();
-               }
-               catch (JobExecutionException e) {
-                       Throwable rootCause = e.getCause();
-                       assertTrue(rootCause instanceof IOException);
-                       assertTrue(rootCause.getMessage().contains("broken 
serialization"));
-               }
-               catch (Exception e) {
-                       e.printStackTrace();
-                       fail(e.getMessage());
-               }
-       }
+       public void testIncorrectSerializer3() throws Exception {
+               expectedException.expect(JobExecutionException.class);
+               expectedException.expectCause(
+                       Matchers.both(isA(IOException.class))
+                               .and(hasProperty("message", 
containsString("broken serialization"))));
 
-       @Test
-       public void testIncorrectSerializer3() {
-               try {
-                       ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
-                       env.setParallelism(PARLLELISM);
-                       env.getConfig().disableSysoutLogging();
-
-                       env
-                                       .generateSequence(1, 10 * PARLLELISM)
-                                       .map(new MapFunction<Long, 
ConsumesTooLittle>() {
-                                               @Override
-                                               public ConsumesTooLittle 
map(Long value) throws Exception {
-                                                       return new 
ConsumesTooLittle();
-                                               }
-                                       })
-                                       .rebalance()
-                                       .output(new 
DiscardingOutputFormat<ConsumesTooLittle>());
-
-                       env.execute();
-               }
-               catch (JobExecutionException e) {
-                       Throwable rootCause = e.getCause();
-                       assertTrue(rootCause instanceof IOException);
-                       assertTrue(rootCause.getMessage().contains("broken 
serialization"));
-               }
-               catch (Exception e) {
-                       e.printStackTrace();
-                       fail(e.getMessage());
-               }
+               ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
+               env.setParallelism(PARLLELISM);
+               env.getConfig().disableSysoutLogging();
+
+               env
+                               .generateSequence(1, 10 * PARLLELISM)
+                               .map(new MapFunction<Long, ConsumesTooLittle>() 
{
+                                       @Override
+                                       public ConsumesTooLittle map(Long 
value) throws Exception {
+                                               return new ConsumesTooLittle();
+                                       }
+                               })
+                               .rebalance()
+                               .output(new 
DiscardingOutputFormat<ConsumesTooLittle>());
+
+               env.execute();
        }
 
        @Test
-       public void testIncorrectSerializer4() {
-               try {
-                       ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
-                       env.setParallelism(PARLLELISM);
-                       env.getConfig().disableSysoutLogging();
-
-                       env
-                                       .generateSequence(1, 10 * PARLLELISM)
-                                       .map(new MapFunction<Long, 
ConsumesTooLittleSpanning>() {
-                                               @Override
-                                               public 
ConsumesTooLittleSpanning map(Long value) throws Exception {
-                                                       return new 
ConsumesTooLittleSpanning();
-                                               }
-                                       })
-                                       .rebalance()
-                                       .output(new 
DiscardingOutputFormat<ConsumesTooLittleSpanning>());
-
-                       env.execute();
-               }
-               catch (ProgramInvocationException e) {
-                       Throwable rootCause = e.getCause().getCause();
-                       assertTrue(rootCause instanceof IOException);
-                       assertTrue(rootCause.getMessage().contains("broken 
serialization"));
-               }
-               catch (Exception e) {
-                       e.printStackTrace();
-                       fail(e.getMessage());
-               }
+       public void testIncorrectSerializer4() throws Exception {
 
 Review comment:
   There are two many duplicate codes in above four tests, and only the map 
function is different.
   Is it necessary or feasible to further deduplicate these common codes?

----------------------------------------------------------------
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:
us...@infra.apache.org


> Add sanity checks to SpillingAdaptiveSpanningRecordDeserializer
> ---------------------------------------------------------------
>
>                 Key: FLINK-10356
>                 URL: https://issues.apache.org/jira/browse/FLINK-10356
>             Project: Flink
>          Issue Type: Improvement
>          Components: Network
>    Affects Versions: 1.5.0, 1.5.1, 1.5.2, 1.5.3, 1.5.4, 1.6.0, 1.6.1, 1.7.0
>            Reporter: Nico Kruber
>            Assignee: Nico Kruber
>            Priority: Major
>              Labels: pull-request-available
>
> {{SpillingAdaptiveSpanningRecordDeserializer}} doesn't have any consistency 
> checks for usage calls or serializers behaving properly, e.g. to read only as 
> many bytes as available/promised for that record. At least these checks 
> should be added:
>  # Check that buffers have not been read from yet before adding them (this is 
> an invariant {{SpillingAdaptiveSpanningRecordDeserializer}} works with and 
> from what I can see, it is followed now.
>  # Check that after deserialization, we actually consumed {{recordLength}} 
> bytes
>  ** If not, in the spanning deserializer, we currently simply skip the 
> remaining bytes.
>  ** But in the non-spanning deserializer, we currently continue from the 
> wrong offset.
>  # Protect against {{setNextBuffer}} being called before draining all 
> available records



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to