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.partition;
022    
023    import java.io.IOException;
024    import java.util.ArrayList;
025    import java.util.HashSet;
026    import java.util.Iterator;
027    import java.util.LinkedHashMap;
028    import java.util.List;
029    import java.util.Map;
030    import java.util.Set;
031    
032    import cascading.flow.FlowProcess;
033    import cascading.scheme.Scheme;
034    import cascading.scheme.SinkCall;
035    import cascading.scheme.SourceCall;
036    import cascading.tap.SinkMode;
037    import cascading.tap.Tap;
038    import cascading.tap.TapException;
039    import cascading.tap.type.FileType;
040    import cascading.tuple.Fields;
041    import cascading.tuple.Tuple;
042    import cascading.tuple.TupleEntry;
043    import cascading.tuple.TupleEntryCollector;
044    import cascading.tuple.TupleEntryIterableChainIterator;
045    import cascading.tuple.TupleEntryIterator;
046    import cascading.tuple.TupleEntrySchemeCollector;
047    import cascading.tuple.TupleEntrySchemeIterator;
048    import cascading.tuple.util.TupleViews;
049    import org.slf4j.Logger;
050    import org.slf4j.LoggerFactory;
051    
052    /**
053     *
054     */
055    public abstract class BasePartitionTap<Config, Input, Output> extends Tap<Config, Input, Output>
056      {
057      /** Field LOG */
058      private static final Logger LOG = LoggerFactory.getLogger( BasePartitionTap.class );
059      /** Field OPEN_FILES_THRESHOLD_DEFAULT */
060      protected static final int OPEN_WRITES_THRESHOLD_DEFAULT = 300;
061    
062      private class PartitionIterator extends TupleEntryIterableChainIterator
063        {
064        public PartitionIterator( final FlowProcess<Config> flowProcess, Input input ) throws IOException
065          {
066          super( getSourceFields() );
067    
068          List<Iterator<Tuple>> iterators = new ArrayList<Iterator<Tuple>>();
069    
070          if( input != null )
071            {
072            String identifier = parent.getFullIdentifier( flowProcess );
073            iterators.add( createPartitionEntryIterator( flowProcess, input, identifier, getCurrentIdentifier( flowProcess ) ) );
074            }
075          else
076            {
077            String[] childIdentifiers = getChildPartitionIdentifiers( flowProcess, false );
078    
079            for( String childIdentifier : childIdentifiers )
080              iterators.add( createPartitionEntryIterator( flowProcess, null, parent.getIdentifier(), childIdentifier ) );
081            }
082    
083          reset( iterators );
084          }
085    
086        private PartitionTupleEntryIterator createPartitionEntryIterator( FlowProcess<Config> flowProcess, Input input, String parentIdentifier, String childIdentifier ) throws IOException
087          {
088          TupleEntrySchemeIterator schemeIterator = createTupleEntrySchemeIterator( flowProcess, parent, childIdentifier, input );
089    
090          return new PartitionTupleEntryIterator( getSourceFields(), partition, parentIdentifier, childIdentifier, schemeIterator );
091          }
092        }
093    
094      private class PartitionCollector extends TupleEntryCollector
095        {
096        private final FlowProcess<Config> flowProcess;
097        private final Config conf;
098        private final Fields parentFields;
099        private final Fields partitionFields;
100        private TupleEntry partitionEntry;
101        private final Tuple partitionTuple;
102        private final Tuple parentTuple;
103    
104        public PartitionCollector( FlowProcess<Config> flowProcess )
105          {
106          super( Fields.asDeclaration( getSinkFields() ) );
107          this.flowProcess = flowProcess;
108          this.conf = flowProcess.getConfigCopy();
109          this.parentFields = parent.getSinkFields();
110          this.partitionFields = ( (PartitionScheme) getScheme() ).partitionFields;
111          this.partitionEntry = new TupleEntry( this.partitionFields );
112    
113          this.partitionTuple = TupleViews.createNarrow( getSinkFields().getPos( this.partitionFields ) );
114          this.parentTuple = TupleViews.createNarrow( getSinkFields().getPos( this.parentFields ) );
115    
116          this.partitionEntry.setTuple( partitionTuple );
117          }
118    
119        private TupleEntryCollector getCollector( String path )
120          {
121          TupleEntryCollector collector = collectors.get( path );
122    
123          if( collector != null )
124            return collector;
125    
126          try
127            {
128            LOG.debug( "creating collector for parent: {}, path: {}", parent.getFullIdentifier( conf ), path );
129    
130            collector = createTupleEntrySchemeCollector( flowProcess, parent, path, openedCollectors );
131    
132            openedCollectors++;
133            flowProcess.increment( Counters.Paths_Opened, 1 );
134            }
135          catch( IOException exception )
136            {
137            throw new TapException( "unable to open partition path: " + path, exception );
138            }
139    
140          if( collectors.size() > openWritesThreshold )
141            purgeCollectors();
142    
143          collectors.put( path, collector );
144    
145          if( LOG.isInfoEnabled() && collectors.size() % 100 == 0 )
146            LOG.info( "caching {} open Taps", collectors.size() );
147    
148          return collector;
149          }
150    
151        private void purgeCollectors()
152          {
153          int numToClose = Math.max( 1, (int) ( openWritesThreshold * .10 ) );
154    
155          if( LOG.isInfoEnabled() )
156            LOG.info( "removing {} open Taps from cache of size {}", numToClose, collectors.size() );
157    
158          Set<String> removeKeys = new HashSet<String>();
159          Set<String> keys = collectors.keySet();
160    
161          for( String key : keys )
162            {
163            if( numToClose-- == 0 )
164              break;
165    
166            removeKeys.add( key );
167            }
168    
169          for( String removeKey : removeKeys )
170            closeCollector( collectors.remove( removeKey ) );
171    
172          flowProcess.increment( Counters.Path_Purges, 1 );
173          }
174    
175        @Override
176        public void close()
177          {
178          super.close();
179    
180          try
181            {
182            for( TupleEntryCollector collector : collectors.values() )
183              closeCollector( collector );
184            }
185          finally
186            {
187            collectors.clear();
188            }
189          }
190    
191        private void closeCollector( TupleEntryCollector collector )
192          {
193          if( collector == null )
194            return;
195    
196          try
197            {
198            collector.close();
199    
200            flowProcess.increment( Counters.Paths_Closed, 1 );
201            }
202          catch( Exception exception )
203            {
204            // do nothing
205            }
206          }
207    
208        protected void collect( TupleEntry tupleEntry ) throws IOException
209          {
210          // reset select views
211          TupleViews.reset( partitionTuple, tupleEntry.getTuple() ); // partitionTuple is inside partitionEntry
212          TupleViews.reset( parentTuple, tupleEntry.getTuple() );
213    
214          String path = partition.toPartition( partitionEntry );
215    
216          getCollector( path ).add( parentTuple );
217          }
218        }
219    
220      /** Field parent */
221      protected Tap parent;
222      /** Field partition */
223      protected Partition partition;
224      /** Field keepParentOnDelete */
225      protected boolean keepParentOnDelete = false;
226      /** Field openTapsThreshold */
227      protected int openWritesThreshold = OPEN_WRITES_THRESHOLD_DEFAULT;
228    
229      /** Field openedCollectors */
230      private long openedCollectors = 0;
231      /** Field collectors */
232      private final Map<String, TupleEntryCollector> collectors = new LinkedHashMap<String, TupleEntryCollector>( 1000, .75f, true );
233    
234      protected abstract TupleEntrySchemeCollector createTupleEntrySchemeCollector( FlowProcess<Config> flowProcess, Tap parent, String path, long sequence ) throws IOException;
235    
236      protected abstract TupleEntrySchemeIterator createTupleEntrySchemeIterator( FlowProcess<Config> flowProcess, Tap parent, String path, Input input ) throws IOException;
237    
238      public enum Counters
239        {
240          Paths_Opened, Paths_Closed, Path_Purges
241        }
242    
243      protected BasePartitionTap( Tap parent, Partition partition, int openWritesThreshold )
244        {
245        super( new PartitionScheme( parent.getScheme(), partition.getPartitionFields() ), parent.getSinkMode() );
246        this.parent = parent;
247        this.partition = partition;
248        this.openWritesThreshold = openWritesThreshold;
249        }
250    
251      protected BasePartitionTap( Tap parent, Partition partition, SinkMode sinkMode )
252        {
253        super( new PartitionScheme( parent.getScheme(), partition.getPartitionFields() ), sinkMode );
254        this.parent = parent;
255        this.partition = partition;
256        }
257    
258      protected BasePartitionTap( Tap parent, Partition partition, SinkMode sinkMode, boolean keepParentOnDelete, int openWritesThreshold )
259        {
260        super( new PartitionScheme( parent.getScheme(), partition.getPartitionFields() ), sinkMode );
261        this.parent = parent;
262        this.partition = partition;
263        this.keepParentOnDelete = keepParentOnDelete;
264        this.openWritesThreshold = openWritesThreshold;
265        }
266    
267      /**
268       * Method getParent returns the parent Tap of this PartitionTap object.
269       *
270       * @return the parent (type Tap) of this PartitionTap object.
271       */
272      public Tap getParent()
273        {
274        return parent;
275        }
276    
277      /**
278       * Method getPartition returns the {@link Partition} instance used by this PartitionTap
279       *
280       * @return the partition instance
281       */
282      public Partition getPartition()
283        {
284        return partition;
285        }
286    
287      /**
288       * Method getChildPartitionIdentifiers returns an array of all identifiers for all available partitions.
289       * <p/>
290       * This method is used internally to set all incoming paths, override to limit applicable partitions.
291       * <p/>
292       * Note the returns array may be large.
293       *
294       * @param flowProcess    of type FlowProcess
295       * @param fullyQualified of type boolean
296       * @return a String[] of partition identifiers
297       * @throws IOException
298       */
299      public String[] getChildPartitionIdentifiers( FlowProcess<Config> flowProcess, boolean fullyQualified ) throws IOException
300        {
301        return ( (FileType) parent ).getChildIdentifiers( flowProcess.getConfigCopy(), partition.getPathDepth(), fullyQualified );
302        }
303    
304      @Override
305      public String getIdentifier()
306        {
307        return parent.getIdentifier();
308        }
309    
310      protected abstract String getCurrentIdentifier( FlowProcess<Config> flowProcess );
311    
312      /**
313       * Method getOpenWritesThreshold returns the openTapsThreshold of this PartitionTap object.
314       *
315       * @return the openTapsThreshold (type int) of this PartitionTap object.
316       */
317      public int getOpenWritesThreshold()
318        {
319        return openWritesThreshold;
320        }
321    
322      @Override
323      public TupleEntryCollector openForWrite( FlowProcess<Config> flowProcess, Output output ) throws IOException
324        {
325        return new PartitionCollector( flowProcess );
326        }
327    
328      @Override
329      public TupleEntryIterator openForRead( FlowProcess<Config> flowProcess, Input input ) throws IOException
330        {
331        return new PartitionIterator( flowProcess, input );
332        }
333    
334      @Override
335      public boolean createResource( Config conf ) throws IOException
336        {
337        return parent.createResource( conf );
338        }
339    
340      @Override
341      public boolean deleteResource( Config conf ) throws IOException
342        {
343        return keepParentOnDelete || parent.deleteResource( conf );
344        }
345    
346      @Override
347      public boolean commitResource( Config conf ) throws IOException
348        {
349        return parent.commitResource( conf );
350        }
351    
352      @Override
353      public boolean rollbackResource( Config conf ) throws IOException
354        {
355        return parent.rollbackResource( conf );
356        }
357    
358      @Override
359      public boolean resourceExists( Config conf ) throws IOException
360        {
361        return parent.resourceExists( conf );
362        }
363    
364      @Override
365      public long getModifiedTime( Config conf ) throws IOException
366        {
367        return parent.getModifiedTime( conf );
368        }
369    
370      @Override
371      public boolean equals( Object object )
372        {
373        if( this == object )
374          return true;
375        if( object == null || getClass() != object.getClass() )
376          return false;
377        if( !super.equals( object ) )
378          return false;
379    
380        BasePartitionTap that = (BasePartitionTap) object;
381    
382        if( parent != null ? !parent.equals( that.parent ) : that.parent != null )
383          return false;
384        if( partition != null ? !partition.equals( that.partition ) : that.partition != null )
385          return false;
386    
387        return true;
388        }
389    
390      @Override
391      public int hashCode()
392        {
393        int result = super.hashCode();
394        result = 31 * result + ( parent != null ? parent.hashCode() : 0 );
395        result = 31 * result + ( partition != null ? partition.hashCode() : 0 );
396        return result;
397        }
398    
399      @Override
400      public String toString()
401        {
402        return getClass().getSimpleName() + "[\"" + parent + "\"]" + "[\"" + partition + "\"]";
403        }
404    
405      public static class PartitionScheme<Config, Input, Output> extends Scheme<Config, Input, Output, Void, Void>
406        {
407        private final Scheme scheme;
408        private final Fields partitionFields;
409    
410        public PartitionScheme( Scheme scheme )
411          {
412          this.scheme = scheme;
413          this.partitionFields = null;
414          }
415    
416        public PartitionScheme( Scheme scheme, Fields partitionFields )
417          {
418          this.scheme = scheme;
419    
420          if( partitionFields == null || partitionFields.isAll() )
421            this.partitionFields = null;
422          else if( partitionFields.isDefined() )
423            this.partitionFields = partitionFields;
424          else
425            throw new IllegalArgumentException( "partitionFields must be defined or the ALL substitution, got: " + partitionFields.printVerbose() );
426          }
427    
428        public Fields getSinkFields()
429          {
430          if( partitionFields == null || scheme.getSinkFields().isAll() )
431            return scheme.getSinkFields();
432    
433          return Fields.merge( scheme.getSinkFields(), partitionFields );
434          }
435    
436        public void setSinkFields( Fields sinkFields )
437          {
438          scheme.setSinkFields( sinkFields );
439          }
440    
441        public Fields getSourceFields()
442          {
443          if( partitionFields == null || scheme.getSourceFields().isUnknown() )
444            return scheme.getSourceFields();
445    
446          return Fields.merge( scheme.getSourceFields(), partitionFields );
447          }
448    
449        public void setSourceFields( Fields sourceFields )
450          {
451          scheme.setSourceFields( sourceFields );
452          }
453    
454        public int getNumSinkParts()
455          {
456          return scheme.getNumSinkParts();
457          }
458    
459        public void setNumSinkParts( int numSinkParts )
460          {
461          scheme.setNumSinkParts( numSinkParts );
462          }
463    
464        @Override
465        public void sourceConfInit( FlowProcess<Config> flowProcess, Tap<Config, Input, Output> tap, Config conf )
466          {
467          scheme.sourceConfInit( flowProcess, tap, conf );
468          }
469    
470        @Override
471        public void sourcePrepare( FlowProcess<Config> flowProcess, SourceCall<Void, Input> sourceCall ) throws IOException
472          {
473          scheme.sourcePrepare( flowProcess, sourceCall );
474          }
475    
476        @Override
477        public boolean source( FlowProcess<Config> flowProcess, SourceCall<Void, Input> sourceCall ) throws IOException
478          {
479          throw new UnsupportedOperationException( "should never be called" );
480          }
481    
482        @Override
483        public void sourceCleanup( FlowProcess<Config> flowProcess, SourceCall<Void, Input> sourceCall ) throws IOException
484          {
485          scheme.sourceCleanup( flowProcess, sourceCall );
486          }
487    
488        @Override
489        public void sinkConfInit( FlowProcess<Config> flowProcess, Tap<Config, Input, Output> tap, Config conf )
490          {
491          scheme.sinkConfInit( flowProcess, tap, conf );
492          }
493    
494        @Override
495        public void sinkPrepare( FlowProcess<Config> flowProcess, SinkCall<Void, Output> sinkCall ) throws IOException
496          {
497          scheme.sinkPrepare( flowProcess, sinkCall );
498          }
499    
500        @Override
501        public void sink( FlowProcess<Config> flowProcess, SinkCall<Void, Output> sinkCall ) throws IOException
502          {
503          throw new UnsupportedOperationException( "should never be called" );
504          }
505    
506        @Override
507        public void sinkCleanup( FlowProcess<Config> flowProcess, SinkCall<Void, Output> sinkCall ) throws IOException
508          {
509          scheme.sinkCleanup( flowProcess, sinkCall );
510          }
511        }
512      }