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 }