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.Closeable;
024import java.io.IOException;
025
026import cascading.flow.FlowProcess;
027import cascading.flow.hadoop.MapRed;
028import cascading.flow.hadoop.util.HadoopUtil;
029import cascading.tap.Tap;
030import cascading.tap.TapException;
031import cascading.tap.hadoop.util.Hadoop18TapUtil;
032import org.apache.hadoop.conf.Configuration;
033import org.apache.hadoop.fs.Path;
034import org.apache.hadoop.mapred.FileOutputFormat;
035import org.apache.hadoop.mapred.OutputCollector;
036import org.apache.hadoop.mapred.OutputFormat;
037import org.apache.hadoop.mapred.RecordReader;
038import org.apache.hadoop.mapred.RecordWriter;
039import org.apache.hadoop.mapred.Reporter;
040import org.slf4j.Logger;
041import org.slf4j.LoggerFactory;
042
043import static cascading.flow.hadoop.util.HadoopUtil.asJobConfInstance;
044
045/**
046 *
047 */
048public class TapOutputCollector implements OutputCollector, Closeable
049  {
050  private static final Logger LOG = LoggerFactory.getLogger( TapOutputCollector.class );
051
052  public static final String PART_TASK_PATTERN = "%s%spart-%05d";
053  public static final String PART_TASK_SEQ_PATTERN = "%s%spart-%05d-%05d";
054
055  /** Field conf */
056  private Configuration conf;
057  /** Field writer */
058  private RecordWriter writer;
059  /** Field filenamePattern */
060  private String filenamePattern;
061  /** Field filename */
062  private String filename;
063  /** Field tap */
064  private Tap<Configuration, RecordReader, OutputCollector> tap;
065  /** Field prefix */
066  private String prefix;
067  /** Field sequence */
068  private long sequence;
069  /** Field isFileOutputFormat */
070  private boolean isFileOutputFormat;
071  private final FlowProcess<? extends Configuration> flowProcess;
072
073  public TapOutputCollector( FlowProcess<? extends Configuration> flowProcess, Tap<Configuration, RecordReader, OutputCollector> tap ) throws IOException
074    {
075    this( flowProcess, tap, null );
076    }
077
078  public TapOutputCollector( FlowProcess<? extends Configuration> flowProcess, Tap<Configuration, RecordReader, OutputCollector> tap, String prefix ) throws IOException
079    {
080    this( flowProcess, tap, prefix, -1 );
081    }
082
083  public TapOutputCollector( FlowProcess<? extends Configuration> flowProcess, Tap<Configuration, RecordReader, OutputCollector> tap, String prefix, long sequence ) throws IOException
084    {
085    this.tap = tap;
086    this.sequence = sequence;
087    this.prefix = prefix == null || prefix.length() == 0 ? null : prefix;
088    this.flowProcess = flowProcess;
089    this.conf = this.flowProcess.getConfigCopy();
090    this.filenamePattern = this.conf.get( "cascading.tapcollector.partname", sequence == -1 ? PART_TASK_PATTERN : PART_TASK_SEQ_PATTERN );
091
092    initialize();
093    }
094
095  protected void initialize() throws IOException
096    {
097    tap.sinkConfInit( flowProcess, conf );
098
099    OutputFormat outputFormat = asJobConfInstance( conf ).getOutputFormat();
100
101    // todo: use OutputCommitter class
102
103    isFileOutputFormat = outputFormat instanceof FileOutputFormat;
104
105    if( isFileOutputFormat )
106      {
107      Hadoop18TapUtil.setupJob( conf );
108      Hadoop18TapUtil.setupTask( conf );
109
110      int partition = conf.getInt( "mapred.task.partition", conf.getInt( "mapreduce.task.partition", 0 ) );
111
112      long localSequence = sequence == -1 ? 0 : sequence;
113
114      if( prefix != null )
115        filename = String.format( filenamePattern, prefix, "/", partition, localSequence );
116      else
117        filename = String.format( filenamePattern, "", "", partition, localSequence );
118      }
119
120    LOG.info( "creating path: {}", filename );
121
122    writer = outputFormat.getRecordWriter( null, asJobConfInstance( conf ), filename, getReporter() );
123    }
124
125  private Reporter getReporter()
126    {
127    Reporter reporter = Reporter.NULL;
128
129    if( flowProcess instanceof MapRed )
130      reporter = ( (MapRed) flowProcess ).getReporter(); // may return Reporter.NULL
131
132    return reporter;
133    }
134
135  /**
136   * Method collect writes the given values to the {@link Tap} this instance encapsulates.
137   *
138   * @param writableComparable of type WritableComparable
139   * @param writable           of type Writable
140   * @throws IOException when
141   */
142  public void collect( Object writableComparable, Object writable ) throws IOException
143    {
144    flowProcess.keepAlive();
145    writer.write( writableComparable, writable );
146    }
147
148  public void close()
149    {
150    try
151      {
152      if( isFileOutputFormat )
153        LOG.info( "closing tap collector for: {}", new Path( tap.getIdentifier(), filename ) );
154      else
155        LOG.info( "closing tap collector for: {}", tap );
156
157      try
158        {
159        writer.close( getReporter() );
160        }
161      finally
162        {
163        if( isFileOutputFormat )
164          {
165          boolean needsTaskCommit = Hadoop18TapUtil.needsTaskCommit( conf );
166
167          boolean cleanJob = true;
168
169          if( needsTaskCommit )
170            cleanJob = Hadoop18TapUtil.commitTask( conf );
171
172          if( cleanJob ) // don't delete _temporary if still contents
173            Hadoop18TapUtil.cleanupJob( conf );
174
175          if( !HadoopUtil.isInflow( conf ) )
176            Hadoop18TapUtil.writeSuccessMarker( conf );
177          }
178        }
179      }
180    catch( IOException exception )
181      {
182      LOG.warn( "exception closing: {}", filename, exception );
183      throw new TapException( "exception closing: " + filename, exception );
184      }
185    }
186  }