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

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

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1625#discussion_r57136323
  
    --- Diff: 
flink-java/src/test/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormatTest.java
 ---
    @@ -0,0 +1,216 @@
    +/*
    + * 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.api.java.hadoop.mapred;
    +
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.hadoop.conf.Configurable;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.mapred.*;
    +import org.apache.hadoop.util.Progressable;
    +import org.junit.Test;
    +import org.mockito.Matchers;
    +
    +import java.io.IOException;
    +
    +import static org.mockito.Matchers.anyString;
    +import static org.mockito.Mockito.*;
    +
    +public class HadoopOutputFormatTest {
    +
    +    @Test
    +    public void testOpen() throws Exception {
    +
    +        OutputFormat<String, Long> dummyOutputFormat = 
mock(DummyOutputFormat.class);
    +        DummyOutputCommitter outputCommitter = 
mock(DummyOutputCommitter.class);
    +        JobConf jobConf = mock(JobConf.class);
    +        when(jobConf.getOutputCommitter()).thenReturn(outputCommitter);
    +
    +        HadoopOutputFormat<String, Long> outputFormat = new 
HadoopOutputFormat<>(dummyOutputFormat, jobConf);
    +
    +        outputFormat.open(1, 1);
    +
    +        verify(jobConf, times(1)).getOutputCommitter();
    +        verify(outputCommitter, times(1)).setupJob(any(JobContext.class));
    +        verify(dummyOutputFormat, 
times(1)).getRecordWriter(any(FileSystem.class), any(JobConf.class), 
anyString(), any(Progressable.class));
    +    }
    +
    +    @Test
    +    public void testConfigureWithConfigurable() {
    +        ConfigurableDummyOutputFormat dummyOutputFormat = 
mock(ConfigurableDummyOutputFormat.class);
    +        JobConf jobConf = mock(JobConf.class);
    +
    +        HadoopOutputFormat<String, Long> outputFormat = new 
HadoopOutputFormat<>(dummyOutputFormat, jobConf);
    +
    +        
outputFormat.configure(Matchers.<org.apache.flink.configuration.Configuration>any());
    +
    +        verify(dummyOutputFormat, 
times(1)).setConf(any(Configuration.class));
    +    }
    +
    +    @Test
    +    public void testConfigureWithJobConfigurable() {
    +        JobConfigurableDummyOutputFormat dummyOutputFormat = 
mock(JobConfigurableDummyOutputFormat.class);
    +        JobConf jobConf = mock(JobConf.class);
    +
    +        HadoopOutputFormat<String, Long> outputFormat = new 
HadoopOutputFormat<>(dummyOutputFormat, jobConf);
    +
    +        
outputFormat.configure(Matchers.<org.apache.flink.configuration.Configuration>any());
    +
    +        verify(dummyOutputFormat, times(1)).configure(any(JobConf.class));
    +    }
    +
    +    @Test
    +    public void testCloseWithTaskCommit() throws Exception {
    +        OutputFormat<String, Long> dummyOutputFormat = 
mock(DummyOutputFormat.class);
    +        DummyOutputCommitter outputCommitter = 
mock(DummyOutputCommitter.class);
    +        
when(outputCommitter.needsTaskCommit(any(TaskAttemptContext.class))).thenReturn(true);
    +        DummyRecordWriter recordWriter = mock(DummyRecordWriter.class);
    +        JobConf jobConf = mock(JobConf.class);
    +
    +        HadoopOutputFormat<String, Long> outputFormat = new 
HadoopOutputFormat<>(dummyOutputFormat, jobConf);
    +        outputFormat.recordWriter = recordWriter;
    +        outputFormat.outputCommitter = outputCommitter;
    +
    +        outputFormat.close();
    +
    +        verify(recordWriter, times(1)).close(any(Reporter.class));
    +        verify(outputCommitter, 
times(1)).commitTask(any(TaskAttemptContext.class));
    +    }
    +
    +    @Test
    +    public void testCloseWithoutTaskCommit() throws Exception {
    +        OutputFormat<String, Long> dummyOutputFormat = 
mock(DummyOutputFormat.class);
    +        DummyOutputCommitter outputCommitter = 
mock(DummyOutputCommitter.class);
    +        
when(outputCommitter.needsTaskCommit(any(TaskAttemptContext.class))).thenReturn(false);
    +        DummyRecordWriter recordWriter = mock(DummyRecordWriter.class);
    +        JobConf jobConf = mock(JobConf.class);
    +
    +        HadoopOutputFormat<String, Long> outputFormat = new 
HadoopOutputFormat<>(dummyOutputFormat, jobConf);
    +        outputFormat.recordWriter = recordWriter;
    +        outputFormat.outputCommitter = outputCommitter;
    +
    +        outputFormat.close();
    +
    +        verify(recordWriter, times(1)).close(any(Reporter.class));
    +        verify(outputCommitter, 
times(0)).commitTask(any(TaskAttemptContext.class));
    +    }
    +
    +    @Test
    +    public void testWriteRecord() throws Exception {
    +        OutputFormat<String, Long> dummyOutputFormat = 
mock(DummyOutputFormat.class);
    +        DummyRecordWriter recordWriter = mock(DummyRecordWriter.class);
    +        JobConf jobConf = mock(JobConf.class);
    +
    +        HadoopOutputFormat<String, Long> outputFormat = new 
HadoopOutputFormat<>(dummyOutputFormat, jobConf);
    +        outputFormat.recordWriter = recordWriter;
    +
    +        outputFormat.writeRecord(new Tuple2<>("key", 1L));
    +
    +        verify(recordWriter, times(1)).write(anyString(), anyLong());
    +    }
    +
    +    @Test
    +    public void testFinalizeGlobal() throws Exception {
    +        OutputFormat<String, Long> dummyOutputFormat = 
mock(DummyOutputFormat.class);
    +        DummyOutputCommitter outputCommitter = 
mock(DummyOutputCommitter.class);
    +        JobConf jobConf = mock(JobConf.class);
    --- End diff --
    
    This line needs to be replaced by `JobConf jobConf = spy(new JobConf());`.
    
    The Hadoop code copies the `JobConf` and access additional fields that 
cannot be mocked.


> Add tests for HadoopOutputFormats
> ---------------------------------
>
>                 Key: FLINK-2445
>                 URL: https://issues.apache.org/jira/browse/FLINK-2445
>             Project: Flink
>          Issue Type: Test
>          Components: Hadoop Compatibility, Tests
>    Affects Versions: 0.9.1, 0.10.0
>            Reporter: Fabian Hueske
>            Assignee: Martin Liesenberg
>              Labels: starter
>
> The HadoopOutputFormats and HadoopOutputFormatBase classes are not 
> sufficiently covered by unit tests.
> We need tests that ensure that the methods of the wrapped Hadoop 
> OutputFormats are correctly called. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to