001/*
002 * Copyright (c) 2016-2018 Chris K Wensel <chris@wensel.net>. All Rights Reserved.
003 * Copyright (c) 2007-2017 Xplenty, Inc. All Rights Reserved.
004 *
005 * Project and contact information: http://www.cascading.org/
006 *
007 * This file is part of the Cascading project.
008 *
009 * Licensed under the Apache License, Version 2.0 (the "License");
010 * you may not use this file except in compliance with the License.
011 * You may obtain a copy of the License at
012 *
013 *     http://www.apache.org/licenses/LICENSE-2.0
014 *
015 * Unless required by applicable law or agreed to in writing, software
016 * distributed under the License is distributed on an "AS IS" BASIS,
017 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
018 * See the License for the specific language governing permissions and
019 * limitations under the License.
020 */
021
022package cascading.scheme.hadoop;
023
024import java.beans.ConstructorProperties;
025import java.io.IOException;
026import java.nio.charset.Charset;
027import java.util.regex.Pattern;
028
029import cascading.flow.FlowProcess;
030import cascading.flow.hadoop.util.HadoopUtil;
031import cascading.management.annotation.Property;
032import cascading.management.annotation.PropertyDescription;
033import cascading.management.annotation.Visibility;
034import cascading.scheme.FileFormat;
035import cascading.scheme.Scheme;
036import cascading.scheme.SinkCall;
037import cascading.scheme.SourceCall;
038import cascading.tap.Tap;
039import cascading.tuple.Fields;
040import cascading.tuple.Tuple;
041import cascading.tuple.TupleEntry;
042import org.apache.hadoop.conf.Configuration;
043import org.apache.hadoop.io.LongWritable;
044import org.apache.hadoop.io.Text;
045import org.apache.hadoop.mapred.InputFormat;
046import org.apache.hadoop.mapred.JobConf;
047import org.apache.hadoop.mapred.OutputCollector;
048import org.apache.hadoop.mapred.OutputFormat;
049import org.apache.hadoop.mapred.RecordReader;
050import org.apache.hadoop.mapred.TextInputFormat;
051import org.apache.hadoop.mapred.TextOutputFormat;
052
053import static cascading.flow.hadoop.util.HadoopUtil.asJobConfInstance;
054
055/**
056 * A TextLine is a type of {@link cascading.scheme.Scheme} for plain text files. Files are broken into
057 * lines. Either line-feed or carriage-return are used to signal end of line.
058 * <p>
059 * By default, this scheme returns a {@link Tuple} with two fields, "offset" and "line".
060 * <p>
061 * Many of the constructors take both "sourceFields" and "sinkFields". sourceFields denote the field names
062 * to be used instead of the names "offset" and "line". sinkFields is a selector and is by default {@link Fields#ALL}.
063 * Any available field names can be given if only a subset of the incoming fields should be used.
064 * <p>
065 * If a {@link Fields} instance is passed on the constructor as sourceFields having only one field, the return tuples
066 * will simply be the "line" value using the given field name.
067 * <p>
068 * Note that TextLine will concatenate all the Tuple values for the selected fields with a TAB delimiter before
069 * writing out the line.
070 * <p>
071 * Note sink compression is {@link Compress#DISABLE} by default. If {@code null} is passed to the constructor
072 * for the compression value, it will remain disabled.
073 * <p>
074 * If any of the input files end with ".zip", an error will be thrown.
075 * * <p>
076 * By default, all text is encoded/decoded as UTF-8. This can be changed via the {@code charsetName} constructor
077 * argument.
078 */
079public class TextLine extends Scheme<Configuration, RecordReader, OutputCollector, Object[], Object[]> implements FileFormat
080  {
081  public enum Compress
082    {
083      DEFAULT, ENABLE, DISABLE
084    }
085
086  public static final String DEFAULT_CHARSET = "UTF-8";
087
088  /** Field serialVersionUID */
089  private static final long serialVersionUID = 1L;
090  /** Field DEFAULT_SOURCE_FIELDS */
091  public static final Fields DEFAULT_SOURCE_FIELDS = new Fields( "offset", "line" ).applyTypes( Long.TYPE, String.class );
092
093  /** Field zipPattern */
094  private static final Pattern zipPattern = Pattern.compile( "\\.[zZ][iI][pP]([ ,]|$)" );
095
096  /** Field sinkCompression */
097  Compress sinkCompression = Compress.DISABLE;
098
099  /** Field charsetName */
100  String charsetName = DEFAULT_CHARSET;
101
102  /**
103   * Creates a new TextLine instance that sources "offset" and "line" fields, and sinks all incoming fields, where
104   * "offset" is the byte offset in the input file.
105   */
106  public TextLine()
107    {
108    super( DEFAULT_SOURCE_FIELDS );
109    }
110
111  /**
112   * Creates a new TextLine instance that sources "offset" and "line" fields, and sinks all incoming fields, where
113   * "offset" is the byte offset in the input file.
114   *
115   * @param numSinkParts of type int
116   */
117  @ConstructorProperties({"numSinkParts"})
118  public TextLine( int numSinkParts )
119    {
120    super( DEFAULT_SOURCE_FIELDS, numSinkParts );
121    }
122
123  /**
124   * Creates a new TextLine instance that sources "offset" and "line" fields, and sinks all incoming fields, where
125   * "offset" is the byte offset in the input file.
126   *
127   * @param sinkCompression of type Compress
128   */
129  @ConstructorProperties({"sinkCompression"})
130  public TextLine( Compress sinkCompression )
131    {
132    super( DEFAULT_SOURCE_FIELDS );
133
134    setSinkCompression( sinkCompression );
135    }
136
137  /**
138   * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
139   * subsequent tuples.
140   *
141   * @param sourceFields the source fields for this scheme
142   * @param sinkFields   the sink fields for this scheme
143   */
144  @ConstructorProperties({"sourceFields", "sinkFields"})
145  public TextLine( Fields sourceFields, Fields sinkFields )
146    {
147    super( sourceFields, sinkFields );
148
149    verify( sourceFields );
150    }
151
152  /**
153   * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
154   * subsequent tuples.
155   *
156   * @param sourceFields the source fields for this scheme
157   * @param sinkFields   the sink fields for this scheme
158   * @param charsetName  of type String
159   */
160  @ConstructorProperties({"sourceFields", "sinkFields", "charsetName"})
161  public TextLine( Fields sourceFields, Fields sinkFields, String charsetName )
162    {
163    super( sourceFields, sinkFields );
164
165    // throws an exception if not found
166    setCharsetName( charsetName );
167
168    verify( sourceFields );
169    }
170
171  /**
172   * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
173   * subsequent tuples.
174   *
175   * @param sourceFields the source fields for this scheme
176   * @param sinkFields   the sink fields for this scheme
177   * @param numSinkParts of type int
178   */
179  @ConstructorProperties({"sourceFields", "sinkFields", "numSinkParts"})
180  public TextLine( Fields sourceFields, Fields sinkFields, int numSinkParts )
181    {
182    super( sourceFields, sinkFields, numSinkParts );
183
184    verify( sourceFields );
185    }
186
187  /**
188   * Constructor TextLine creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
189   * subsequent tuples.
190   *
191   * @param sourceFields    of type Fields
192   * @param sinkFields      of type Fields
193   * @param sinkCompression of type Compress
194   */
195  @ConstructorProperties({"sourceFields", "sinkFields", "sinkCompression"})
196  public TextLine( Fields sourceFields, Fields sinkFields, Compress sinkCompression )
197    {
198    super( sourceFields, sinkFields );
199
200    setSinkCompression( sinkCompression );
201
202    verify( sourceFields );
203    }
204
205  /**
206   * Constructor TextLine creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
207   * subsequent tuples.
208   *
209   * @param sourceFields    of type Fields
210   * @param sinkFields      of type Fields
211   * @param sinkCompression of type Compress
212   * @param charsetName     of type String
213   */
214  @ConstructorProperties({"sourceFields", "sinkFields", "sinkCompression", "charsetName"})
215  public TextLine( Fields sourceFields, Fields sinkFields, Compress sinkCompression, String charsetName )
216    {
217    super( sourceFields, sinkFields );
218
219    setSinkCompression( sinkCompression );
220
221    // throws an exception if not found
222    setCharsetName( charsetName );
223
224    verify( sourceFields );
225    }
226
227  /**
228   * Constructor TextLine creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
229   * subsequent tuples.
230   *
231   * @param sourceFields    of type Fields
232   * @param sinkFields      of type Fields
233   * @param sinkCompression of type Compress
234   * @param numSinkParts    of type int
235   */
236  @ConstructorProperties({"sourceFields", "sinkFields", "sinkCompression", "numSinkParts"})
237  public TextLine( Fields sourceFields, Fields sinkFields, Compress sinkCompression, int numSinkParts )
238    {
239    super( sourceFields, sinkFields, numSinkParts );
240
241    setSinkCompression( sinkCompression );
242
243    verify( sourceFields );
244    }
245
246  /**
247   * Constructor TextLine creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
248   * subsequent tuples.
249   *
250   * @param sourceFields    of type Fields
251   * @param sinkFields      of type Fields
252   * @param sinkCompression of type Compress
253   * @param numSinkParts    of type int
254   * @param charsetName     of type String
255   */
256  @ConstructorProperties({"sourceFields", "sinkFields", "sinkCompression", "numSinkParts", "charsetName"})
257  public TextLine( Fields sourceFields, Fields sinkFields, Compress sinkCompression, int numSinkParts, String charsetName )
258    {
259    super( sourceFields, sinkFields, numSinkParts );
260
261    setSinkCompression( sinkCompression );
262
263    // throws an exception if not found
264    setCharsetName( charsetName );
265
266    verify( sourceFields );
267    }
268
269  /**
270   * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
271   * subsequent tuples.
272   *
273   * @param sourceFields the source fields for this scheme
274   */
275  @ConstructorProperties({"sourceFields"})
276  public TextLine( Fields sourceFields )
277    {
278    super( sourceFields );
279
280    verify( sourceFields );
281    }
282
283  /**
284   * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
285   * subsequent tuples.
286   *
287   * @param sourceFields the source fields for this scheme
288   * @param charsetName  of type String
289   */
290  @ConstructorProperties({"sourceFields", "charsetName"})
291  public TextLine( Fields sourceFields, String charsetName )
292    {
293    super( sourceFields );
294
295    // throws an exception if not found
296    setCharsetName( charsetName );
297
298    verify( sourceFields );
299    }
300
301  /**
302   * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the
303   * subsequent tuples. The resulting data set will have numSinkParts.
304   *
305   * @param sourceFields the source fields for this scheme
306   * @param numSinkParts of type int
307   */
308  @ConstructorProperties({"sourceFields", "numSinkParts"})
309  public TextLine( Fields sourceFields, int numSinkParts )
310    {
311    super( sourceFields, numSinkParts );
312
313    verify( sourceFields );
314    }
315
316  protected void setCharsetName( String charsetName )
317    {
318    if( charsetName != null )
319      this.charsetName = charsetName;
320
321    Charset.forName( this.charsetName );
322    }
323
324  @Property(name = "charset", visibility = Visibility.PUBLIC)
325  @PropertyDescription(value = "character set used in this scheme.")
326  public String getCharsetName()
327    {
328    return charsetName;
329    }
330
331  protected void verify( Fields sourceFields )
332    {
333    if( sourceFields.size() < 1 || sourceFields.size() > 2 )
334      throw new IllegalArgumentException( "this scheme requires either one or two source fields, given [" + sourceFields + "]" );
335    }
336
337  /**
338   * Method getSinkCompression returns the sinkCompression of this TextLine object.
339   *
340   * @return the sinkCompression (type Compress) of this TextLine object.
341   */
342  @Property(name = "sinkCompression", visibility = Visibility.PUBLIC)
343  @PropertyDescription(value = "The compression of the scheme when used in a sink.")
344  public Compress getSinkCompression()
345    {
346    return sinkCompression;
347    }
348
349  /**
350   * Method setSinkCompression sets the sinkCompression of this TextLine object. If null, compression will remain disabled.
351   *
352   * @param sinkCompression the sinkCompression of this TextLine object.
353   */
354  public void setSinkCompression( Compress sinkCompression )
355    {
356    if( sinkCompression != null ) // leave disabled if null
357      this.sinkCompression = sinkCompression;
358    }
359
360  @Override
361  public void sourceConfInit( FlowProcess<? extends Configuration> flowProcess, Tap<Configuration, RecordReader, OutputCollector> tap, Configuration conf )
362    {
363    JobConf jobConf = asJobConfInstance( conf );
364    String paths = jobConf.get( "mapred.input.dir", "" );
365
366    if( hasZippedFiles( paths ) )
367      throw new IllegalStateException( "cannot read zip files: " + paths );
368
369    conf.setBoolean( "mapred.mapper.new-api", false );
370    conf.setClass( "mapred.input.format.class", TextInputFormat.class, InputFormat.class );
371    }
372
373  private boolean hasZippedFiles( String paths )
374    {
375    if( paths == null || paths.length() == 0 )
376      return false;
377
378    return zipPattern.matcher( paths ).find();
379    }
380
381  @Override
382  public void presentSourceFields( FlowProcess<? extends Configuration> flowProcess, Tap tap, Fields fields )
383    {
384    // do nothing to change TextLine state
385    }
386
387  @Override
388  public void presentSinkFields( FlowProcess<? extends Configuration> flowProcess, Tap tap, Fields fields )
389    {
390    // do nothing to change TextLine state
391    }
392
393  @Override
394  public void sinkConfInit( FlowProcess<? extends Configuration> flowProcess, Tap<Configuration, RecordReader, OutputCollector> tap, Configuration conf )
395    {
396    if( tap.getFullIdentifier( conf ).endsWith( ".zip" ) )
397      throw new IllegalStateException( "cannot write zip files: " + HadoopUtil.getOutputPath( conf ) );
398
399    conf.setBoolean( "mapred.mapper.new-api", false );
400
401    if( getSinkCompression() == Compress.DISABLE )
402      conf.setBoolean( "mapred.output.compress", false );
403    else if( getSinkCompression() == Compress.ENABLE )
404      conf.setBoolean( "mapred.output.compress", true );
405
406    conf.setClass( "mapred.output.key.class", Text.class, Object.class );
407    conf.setClass( "mapred.output.value.class", Text.class, Object.class );
408    conf.setClass( "mapred.output.format.class", TextOutputFormat.class, OutputFormat.class );
409    }
410
411  @Override
412  public void sourcePrepare( FlowProcess<? extends Configuration> flowProcess, SourceCall<Object[], RecordReader> sourceCall )
413    {
414    if( sourceCall.getContext() == null )
415      sourceCall.setContext( new Object[ 3 ] );
416
417    sourceCall.getContext()[ 0 ] = sourceCall.getInput().createKey();
418    sourceCall.getContext()[ 1 ] = sourceCall.getInput().createValue();
419    sourceCall.getContext()[ 2 ] = Charset.forName( charsetName );
420    }
421
422  @Override
423  public boolean source( FlowProcess<? extends Configuration> flowProcess, SourceCall<Object[], RecordReader> sourceCall ) throws IOException
424    {
425    if( !sourceReadInput( sourceCall ) )
426      return false;
427
428    sourceHandleInput( sourceCall );
429
430    return true;
431    }
432
433  private boolean sourceReadInput( SourceCall<Object[], RecordReader> sourceCall ) throws IOException
434    {
435    Object[] context = sourceCall.getContext();
436
437    return sourceCall.getInput().next( context[ 0 ], context[ 1 ] );
438    }
439
440  protected void sourceHandleInput( SourceCall<Object[], RecordReader> sourceCall ) throws IOException
441    {
442    TupleEntry result = sourceCall.getIncomingEntry();
443
444    int index = 0;
445    Object[] context = sourceCall.getContext();
446
447    // coerce into canonical forms
448    if( getSourceFields().size() == 2 )
449      result.setLong( index++, ( (LongWritable) context[ 0 ] ).get() );
450
451    result.setString( index, makeEncodedString( context ) );
452    }
453
454  protected String makeEncodedString( Object[] context )
455    {
456    Text text = (Text) context[ 1 ];
457    return new String( text.getBytes(), 0, text.getLength(), (Charset) context[ 2 ] );
458    }
459
460  @Override
461  public void sourceCleanup( FlowProcess<? extends Configuration> flowProcess, SourceCall<Object[], RecordReader> sourceCall )
462    {
463    sourceCall.setContext( null );
464    }
465
466  @Override
467  public void sinkPrepare( FlowProcess<? extends Configuration> flowProcess, SinkCall<Object[], OutputCollector> sinkCall ) throws IOException
468    {
469    sinkCall.setContext( new Object[ 2 ] );
470
471    sinkCall.getContext()[ 0 ] = new Text();
472    sinkCall.getContext()[ 1 ] = Charset.forName( charsetName );
473    }
474
475  @Override
476  public void sink( FlowProcess<? extends Configuration> flowProcess, SinkCall<Object[], OutputCollector> sinkCall ) throws IOException
477    {
478    Text text = (Text) sinkCall.getContext()[ 0 ];
479    Charset charset = (Charset) sinkCall.getContext()[ 1 ];
480    String line = sinkCall.getOutgoingEntry().getTuple().toString();
481
482    text.set( line.getBytes( charset ) );
483
484    // it's ok to use NULL here so the collector does not write anything
485    sinkCall.getOutput().collect( null, text );
486    }
487
488  @Override
489  public String getExtension()
490    {
491    return "txt";
492    }
493  }