001/*
002 * Copyright (c) 2007-2022 The Cascading Authors. All Rights Reserved.
003 *
004 * Project and contact information: https://cascading.wensel.net/
005 *
006 * This file is part of the Cascading project.
007 *
008 * Licensed under the Apache License, Version 2.0 (the "License");
009 * you may not use this file except in compliance with the License.
010 * You may obtain a copy of the License at
011 *
012 *     http://www.apache.org/licenses/LICENSE-2.0
013 *
014 * Unless required by applicable law or agreed to in writing, software
015 * distributed under the License is distributed on an "AS IS" BASIS,
016 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
017 * See the License for the specific language governing permissions and
018 * limitations under the License.
019 */
020
021package cascading.tap.hadoop.io;
022
023import java.io.IOException;
024import java.util.ArrayList;
025import java.util.Arrays;
026import java.util.Collections;
027import java.util.List;
028import java.util.Map;
029
030import cascading.CascadingException;
031import cascading.flow.hadoop.util.HadoopUtil;
032import cascading.util.Util;
033import org.apache.hadoop.fs.Path;
034import org.apache.hadoop.mapred.FileInputFormat;
035import org.apache.hadoop.mapred.InputFormat;
036import org.apache.hadoop.mapred.InputSplit;
037import org.apache.hadoop.mapred.JobConf;
038import org.apache.hadoop.mapred.RecordReader;
039import org.apache.hadoop.mapred.Reporter;
040import org.slf4j.Logger;
041import org.slf4j.LoggerFactory;
042
043/**
044 * Class MultiInputFormat accepts multiple InputFormat class declarations allowing a single MR job
045 * to read data from incompatible file types.
046 */
047public class MultiInputFormat implements InputFormat
048  {
049  /** Field LOG */
050  private static final Logger LOG = LoggerFactory.getLogger( MultiInputFormat.class );
051
052  /**
053   * Used to set the current JobConf with all sub jobs configurations.
054   *
055   * @param toJob
056   * @param fromJobs
057   */
058  public static void addInputFormat( JobConf toJob, JobConf... fromJobs )
059    {
060    toJob.setInputFormat( MultiInputFormat.class );
061    List<Map<String, String>> configs = new ArrayList<Map<String, String>>();
062    List<Path> allPaths = new ArrayList<Path>();
063
064    boolean isLocal = false;
065
066    for( JobConf fromJob : fromJobs )
067      {
068      if( fromJob.get( "mapred.input.format.class" ) == null )
069        throw new CascadingException( "mapred.input.format.class is required, should be set in source Scheme#sourceConfInit" );
070
071      configs.add( HadoopUtil.getConfig( toJob, fromJob ) );
072      Collections.addAll( allPaths, FileInputFormat.getInputPaths( fromJob ) );
073
074      if( !isLocal )
075        isLocal = HadoopUtil.isLocal( fromJob );
076      }
077
078    if( !allPaths.isEmpty() ) // it's possible there aren't any
079      FileInputFormat.setInputPaths( toJob, (Path[]) allPaths.toArray( new Path[ allPaths.size() ] ) );
080
081    try
082      {
083      toJob.set( "cascading.multiinputformats", HadoopUtil.serializeBase64( configs, toJob, true ) );
084      }
085    catch( IOException exception )
086      {
087      throw new CascadingException( "unable to pack input formats", exception );
088      }
089
090    if( isLocal )
091      HadoopUtil.setLocal( toJob );
092    }
093
094  static InputFormat[] getInputFormats( JobConf[] jobConfs )
095    {
096    InputFormat[] inputFormats = new InputFormat[ jobConfs.length ];
097
098    for( int i = 0; i < jobConfs.length; i++ )
099      inputFormats[ i ] = jobConfs[ i ].getInputFormat();
100
101    return inputFormats;
102    }
103
104  private List<Map<String, String>> getConfigs( JobConf job ) throws IOException
105    {
106    return (List<Map<String, String>>)
107      HadoopUtil.deserializeBase64( job.get( "cascading.multiinputformats" ), job, ArrayList.class, true );
108    }
109
110  public void validateInput( JobConf job ) throws IOException
111    {
112    // do nothing, is deprecated
113    }
114
115  /**
116   * Method getSplits delegates to the appropriate InputFormat.
117   *
118   * @param job       of type JobConf
119   * @param numSplits of type int
120   * @return InputSplit[]
121   * @throws IOException when
122   */
123  public InputSplit[] getSplits( JobConf job, int numSplits ) throws IOException
124    {
125    numSplits = numSplits == 0 ? 1 : numSplits;
126
127    List<Map<String, String>> configs = getConfigs( job );
128    JobConf[] jobConfs = HadoopUtil.getJobConfs( job, configs );
129    InputFormat[] inputFormats = getInputFormats( jobConfs );
130
131    // if only one InputFormat, just return what ever it suggests
132    if( inputFormats.length == 1 )
133      return collapse( getSplits( inputFormats, jobConfs, new int[]{numSplits} ), configs );
134
135    int[] indexedSplits = new int[ inputFormats.length ];
136
137    // if we need only a few, the return one for each
138    if( numSplits <= inputFormats.length )
139      {
140      Arrays.fill( indexedSplits, 1 );
141      return collapse( getSplits( inputFormats, jobConfs, indexedSplits ), configs );
142      }
143
144    // attempt to get splits proportionally sized per input format
145    long[] inputSplitSizes = getInputSplitSizes( inputFormats, jobConfs, numSplits );
146    long totalSplitSize = sum( inputSplitSizes );
147
148    if( totalSplitSize == 0 )
149      {
150      Arrays.fill( indexedSplits, 1 );
151      return collapse( getSplits( inputFormats, jobConfs, indexedSplits ), configs );
152      }
153
154    for( int i = 0; i < inputSplitSizes.length; i++ )
155      {
156      int useSplits = (int) Math.ceil( (double) numSplits * inputSplitSizes[ i ] / (double) totalSplitSize );
157      indexedSplits[ i ] = useSplits == 0 ? 1 : useSplits;
158      }
159
160    return collapse( getSplits( inputFormats, jobConfs, indexedSplits ), configs );
161    }
162
163  private long sum( long[] inputSizes )
164    {
165    long size = 0;
166
167    for( long inputSize : inputSizes )
168      size += inputSize;
169
170    return size;
171    }
172
173  private InputSplit[] collapse( InputSplit[][] splits, List<Map<String, String>> configs )
174    {
175    List<InputSplit> splitsList = new ArrayList<InputSplit>();
176
177    for( int i = 0; i < splits.length; i++ )
178      {
179      Map<String, String> config = configs.get( i );
180
181      config.remove( "mapred.input.dir" ); // this is a redundant value, will show up cluster side
182      config.remove( "mapreduce.input.fileinputformat.inputdir" ); // hadoop3
183
184      InputSplit[] split = splits[ i ];
185
186      for( int j = 0; j < split.length; j++ )
187        splitsList.add( new MultiInputSplit( split[ j ], config ) );
188      }
189
190    return splitsList.toArray( new InputSplit[ splitsList.size() ] );
191    }
192
193  private InputSplit[][] getSplits( InputFormat[] inputFormats, JobConf[] jobConfs, int[] numSplits ) throws IOException
194    {
195    InputSplit[][] inputSplits = new InputSplit[ inputFormats.length ][];
196
197    for( int i = 0; i < inputFormats.length; i++ )
198      {
199      inputSplits[ i ] = inputFormats[ i ].getSplits( jobConfs[ i ], numSplits[ i ] );
200
201      // it's reasonable the split array is empty, but really shouldn't be null
202      if( inputSplits[ i ] == null )
203        inputSplits[ i ] = new InputSplit[ 0 ];
204
205      for( int j = 0; j < inputSplits[ i ].length; j++ )
206        {
207        if( inputSplits[ i ][ j ] == null )
208          throw new IllegalStateException( "input format: " + inputFormats[ i ].getClass().getName() + ", returned a split array with nulls" );
209        }
210      }
211
212    return inputSplits;
213    }
214
215  private long[] getInputSplitSizes( InputFormat[] inputFormats, JobConf[] jobConfs, int numSplits ) throws IOException
216    {
217    long[] inputSizes = new long[ inputFormats.length ];
218
219    for( int i = 0; i < inputFormats.length; i++ )
220      {
221      InputFormat inputFormat = inputFormats[ i ];
222      InputSplit[] splits = inputFormat.getSplits( jobConfs[ i ], numSplits );
223
224      inputSizes[ i ] = splits.length;
225      }
226
227    return inputSizes;
228    }
229
230  /**
231   * Method getRecordReader delegates to the appropriate InputFormat.
232   *
233   * @param split    of type InputSplit
234   * @param job      of type JobConf
235   * @param reporter of type Reporter
236   * @return RecordReader
237   * @throws IOException when
238   */
239  public RecordReader getRecordReader( InputSplit split, JobConf job, final Reporter reporter ) throws IOException
240    {
241    final MultiInputSplit multiSplit = (MultiInputSplit) split;
242    final JobConf currentConf = HadoopUtil.mergeConf( job, multiSplit.config, true );
243
244    try
245      {
246      return Util.retry( LOG, 3, 20, "unable to get record reader", new Util.RetryOperator<RecordReader>()
247        {
248
249        @Override
250        public RecordReader operate() throws IOException
251          {
252          return currentConf.getInputFormat().getRecordReader( multiSplit.inputSplit, currentConf, reporter );
253          }
254
255        @Override
256        public boolean rethrow( Exception exception )
257          {
258          return false;
259          }
260        } );
261      }
262    catch( Exception exception )
263      {
264      if( exception instanceof RuntimeException )
265        throw (RuntimeException) exception;
266      else
267        throw (IOException) exception;
268      }
269    }
270  }