001    /*
002     * Copyright (c) 2007-2014 Concurrent, Inc. All Rights Reserved.
003     *
004     * Project and contact information: http://www.cascading.org/
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    
021    package cascading.tap.hadoop;
022    
023    import java.beans.ConstructorProperties;
024    import java.io.IOException;
025    
026    import cascading.flow.FlowProcess;
027    import cascading.tap.SinkMode;
028    import cascading.tap.Tap;
029    import cascading.tap.TapException;
030    import cascading.tap.hadoop.io.HadoopTupleEntrySchemeIterator;
031    import cascading.tap.hadoop.io.MultiInputSplit;
032    import cascading.tap.hadoop.io.TapOutputCollector;
033    import cascading.tap.partition.BasePartitionTap;
034    import cascading.tap.partition.Partition;
035    import cascading.tuple.TupleEntrySchemeCollector;
036    import cascading.tuple.TupleEntrySchemeIterator;
037    import org.apache.hadoop.fs.Path;
038    import org.apache.hadoop.mapred.JobConf;
039    import org.apache.hadoop.mapred.OutputCollector;
040    import org.apache.hadoop.mapred.RecordReader;
041    
042    /**
043     * Class PartitionTap can be used to write tuple streams out to files and sub-directories based on the values in the
044     * current {@link cascading.tuple.Tuple} instance.
045     * <p/>
046     * The constructor takes a {@link cascading.tap.hadoop.Hfs} {@link cascading.tap.Tap} and a {@link Partition}
047     * implementation. This allows Tuple values at given positions to be used as directory names.
048     * <p/>
049     * Note that Hadoop can only sink to directories, and all files in those directories are "part-xxxxx" files.
050     * <p/>
051     * {@code openWritesThreshold} limits the number of open files to be output to. This value defaults to 300 files.
052     * Each time the threshold is exceeded, 10% of the least recently used open files will be closed.
053     * <p/>
054     * PartitionTap will populate a given {@code partition} without regard to case of the values being used. Thus
055     * the resulting paths {@code 2012/June/} and {@code 2012/june/} will likely result in two open files into the same
056     * location. Forcing the case to be consistent with a custom Partition implementation or an upstream
057     * {@link cascading.operation.Function} is recommended, see {@link cascading.operation.expression.ExpressionFunction}.
058     * <p/>
059     * Though Hadoop has no mechanism to prevent simultaneous writes to a directory from multiple jobs, it doesn't mean
060     * its safe to do so. Same is true with the PartitionTap. Interleaving writes to a common parent (root) directory
061     * across multiple flows will very likely lead to data loss.
062     */
063    public class PartitionTap extends BasePartitionTap<JobConf, RecordReader, OutputCollector>
064      {
065      /**
066       * Constructor PartitionTap creates a new PartitionTap instance using the given parent {@link cascading.tap.hadoop.Hfs} Tap as the
067       * base path and default {@link cascading.scheme.Scheme}, and the partition.
068       *
069       * @param parent    of type Tap
070       * @param partition of type String
071       */
072      @ConstructorProperties({"parent", "partition"})
073      public PartitionTap( Hfs parent, Partition partition )
074        {
075        this( parent, partition, OPEN_WRITES_THRESHOLD_DEFAULT );
076        }
077    
078      /**
079       * Constructor PartitionTap creates a new PartitionTap instance using the given parent {@link cascading.tap.hadoop.Hfs} Tap as the
080       * base path and default {@link cascading.scheme.Scheme}, and the partition.
081       * <p/>
082       * {@code openWritesThreshold} limits the number of open files to be output to.
083       *
084       * @param parent              of type Hfs
085       * @param partition           of type String
086       * @param openWritesThreshold of type int
087       */
088      @ConstructorProperties({"parent", "partition", "openWritesThreshold"})
089      public PartitionTap( Hfs parent, Partition partition, int openWritesThreshold )
090        {
091        super( parent, partition, openWritesThreshold );
092        }
093    
094      /**
095       * Constructor PartitionTap creates a new PartitionTap instance using the given parent {@link cascading.tap.hadoop.Hfs} Tap as the
096       * base path and default {@link cascading.scheme.Scheme}, and the partition.
097       *
098       * @param parent    of type Tap
099       * @param partition of type String
100       * @param sinkMode  of type SinkMode
101       */
102      @ConstructorProperties({"parent", "partition", "sinkMode"})
103      public PartitionTap( Hfs parent, Partition partition, SinkMode sinkMode )
104        {
105        super( parent, partition, sinkMode );
106        }
107    
108      /**
109       * Constructor PartitionTap creates a new PartitionTap instance using the given parent {@link cascading.tap.hadoop.Hfs} Tap as the
110       * base path and default {@link cascading.scheme.Scheme}, and the partition.
111       * <p/>
112       * {@code keepParentOnDelete}, when set to true, prevents the parent Tap from being deleted when {@link #deleteResource(Object)}
113       * is called, typically an issue when used inside a {@link cascading.cascade.Cascade}.
114       *
115       * @param parent             of type Tap
116       * @param partition          of type String
117       * @param sinkMode           of type SinkMode
118       * @param keepParentOnDelete of type boolean
119       */
120      @ConstructorProperties({"parent", "partition", "sinkMode", "keepParentOnDelete"})
121      public PartitionTap( Hfs parent, Partition partition, SinkMode sinkMode, boolean keepParentOnDelete )
122        {
123        this( parent, partition, sinkMode, keepParentOnDelete, OPEN_WRITES_THRESHOLD_DEFAULT );
124        }
125    
126      /**
127       * Constructor PartitionTap creates a new PartitionTap instance using the given parent {@link cascading.tap.hadoop.Hfs} Tap as the
128       * base path and default {@link cascading.scheme.Scheme}, and the partition.
129       * <p/>
130       * {@code keepParentOnDelete}, when set to true, prevents the parent Tap from being deleted when {@link #deleteResource(Object)}
131       * is called, typically an issue when used inside a {@link cascading.cascade.Cascade}.
132       * <p/>
133       * {@code openWritesThreshold} limits the number of open files to be output to.
134       *
135       * @param parent              of type Tap
136       * @param partition           of type String
137       * @param sinkMode            of type SinkMode
138       * @param keepParentOnDelete  of type boolean
139       * @param openWritesThreshold of type int
140       */
141      @ConstructorProperties({"parent", "partition", "sinkMode", "keepParentOnDelete", "openWritesThreshold"})
142      public PartitionTap( Hfs parent, Partition partition, SinkMode sinkMode, boolean keepParentOnDelete, int openWritesThreshold )
143        {
144        super( parent, partition, sinkMode, keepParentOnDelete, openWritesThreshold );
145        }
146    
147      @Override
148      protected TupleEntrySchemeCollector createTupleEntrySchemeCollector( FlowProcess<JobConf> flowProcess, Tap parent, String path, long sequence ) throws IOException
149        {
150        TapOutputCollector outputCollector = new TapOutputCollector( flowProcess, parent, path, sequence );
151    
152        return new TupleEntrySchemeCollector<JobConf, OutputCollector>( flowProcess, parent, outputCollector );
153        }
154    
155      @Override
156      protected TupleEntrySchemeIterator createTupleEntrySchemeIterator( FlowProcess<JobConf> flowProcess, Tap parent, String path, RecordReader recordReader ) throws IOException
157        {
158        return new HadoopTupleEntrySchemeIterator( flowProcess, new Hfs( parent.getScheme(), path ), recordReader );
159        }
160    
161      @Override
162      protected String getCurrentIdentifier( FlowProcess<JobConf> flowProcess )
163        {
164        String identifier = flowProcess.getStringProperty( MultiInputSplit.CASCADING_SOURCE_PATH ); // set on current split
165    
166        if( identifier == null )
167          return null;
168    
169        return new Path( identifier ).getParent().toString(); // drop part-xxxx
170        }
171    
172      @Override
173      public void sourceConfInit( FlowProcess<JobConf> flowProcess, JobConf conf )
174        {
175        try
176          {
177          String[] childPartitions = getChildPartitionIdentifiers( flowProcess, true );
178    
179          ( (Hfs) getParent() ).applySourceConfInitIdentifiers( flowProcess, conf, childPartitions );
180          }
181        catch( IOException exception )
182          {
183          throw new TapException( "unable to retrieve child partitions", exception );
184          }
185        }
186      }