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