001 /** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018 019 package org.apache.hadoop.mapreduce.lib.input; 020 021 import java.io.IOException; 022 import java.util.ArrayList; 023 import java.util.Collection; 024 import java.util.LinkedList; 025 import java.util.HashSet; 026 import java.util.List; 027 import java.util.HashMap; 028 import java.util.Set; 029 import java.util.Iterator; 030 import java.util.Map; 031 032 import org.apache.hadoop.classification.InterfaceAudience; 033 import org.apache.hadoop.classification.InterfaceStability; 034 import org.apache.hadoop.conf.Configuration; 035 import org.apache.hadoop.fs.FileSystem; 036 import org.apache.hadoop.fs.FileUtil; 037 import org.apache.hadoop.fs.Path; 038 import org.apache.hadoop.fs.BlockLocation; 039 import org.apache.hadoop.fs.FileStatus; 040 import org.apache.hadoop.fs.PathFilter; 041 import org.apache.hadoop.io.compress.CompressionCodec; 042 import org.apache.hadoop.io.compress.CompressionCodecFactory; 043 import org.apache.hadoop.io.compress.SplittableCompressionCodec; 044 import org.apache.hadoop.mapreduce.InputFormat; 045 import org.apache.hadoop.mapreduce.InputSplit; 046 import org.apache.hadoop.mapreduce.JobContext; 047 import org.apache.hadoop.mapreduce.RecordReader; 048 import org.apache.hadoop.mapreduce.TaskAttemptContext; 049 import org.apache.hadoop.net.NodeBase; 050 import org.apache.hadoop.net.NetworkTopology; 051 052 /** 053 * An abstract {@link InputFormat} that returns {@link CombineFileSplit}'s in 054 * {@link InputFormat#getSplits(JobContext)} method. 055 * 056 * Splits are constructed from the files under the input paths. 057 * A split cannot have files from different pools. 058 * Each split returned may contain blocks from different files. 059 * If a maxSplitSize is specified, then blocks on the same node are 060 * combined to form a single split. Blocks that are left over are 061 * then combined with other blocks in the same rack. 062 * If maxSplitSize is not specified, then blocks from the same rack 063 * are combined in a single split; no attempt is made to create 064 * node-local splits. 065 * If the maxSplitSize is equal to the block size, then this class 066 * is similar to the default splitting behavior in Hadoop: each 067 * block is a locally processed split. 068 * Subclasses implement 069 * {@link InputFormat#createRecordReader(InputSplit, TaskAttemptContext)} 070 * to construct <code>RecordReader</code>'s for 071 * <code>CombineFileSplit</code>'s. 072 * 073 * @see CombineFileSplit 074 */ 075 @InterfaceAudience.Public 076 @InterfaceStability.Stable 077 public abstract class CombineFileInputFormat<K, V> 078 extends FileInputFormat<K, V> { 079 080 public static final String SPLIT_MINSIZE_PERNODE = 081 "mapreduce.input.fileinputformat.split.minsize.per.node"; 082 public static final String SPLIT_MINSIZE_PERRACK = 083 "mapreduce.input.fileinputformat.split.minsize.per.rack"; 084 // ability to limit the size of a single split 085 private long maxSplitSize = 0; 086 private long minSplitSizeNode = 0; 087 private long minSplitSizeRack = 0; 088 089 // A pool of input paths filters. A split cannot have blocks from files 090 // across multiple pools. 091 private ArrayList<MultiPathFilter> pools = new ArrayList<MultiPathFilter>(); 092 093 // mapping from a rack name to the set of Nodes in the rack 094 private HashMap<String, Set<String>> rackToNodes = 095 new HashMap<String, Set<String>>(); 096 /** 097 * Specify the maximum size (in bytes) of each split. Each split is 098 * approximately equal to the specified size. 099 */ 100 protected void setMaxSplitSize(long maxSplitSize) { 101 this.maxSplitSize = maxSplitSize; 102 } 103 104 /** 105 * Specify the minimum size (in bytes) of each split per node. 106 * This applies to data that is left over after combining data on a single 107 * node into splits that are of maximum size specified by maxSplitSize. 108 * This leftover data will be combined into its own split if its size 109 * exceeds minSplitSizeNode. 110 */ 111 protected void setMinSplitSizeNode(long minSplitSizeNode) { 112 this.minSplitSizeNode = minSplitSizeNode; 113 } 114 115 /** 116 * Specify the minimum size (in bytes) of each split per rack. 117 * This applies to data that is left over after combining data on a single 118 * rack into splits that are of maximum size specified by maxSplitSize. 119 * This leftover data will be combined into its own split if its size 120 * exceeds minSplitSizeRack. 121 */ 122 protected void setMinSplitSizeRack(long minSplitSizeRack) { 123 this.minSplitSizeRack = minSplitSizeRack; 124 } 125 126 /** 127 * Create a new pool and add the filters to it. 128 * A split cannot have files from different pools. 129 */ 130 protected void createPool(List<PathFilter> filters) { 131 pools.add(new MultiPathFilter(filters)); 132 } 133 134 /** 135 * Create a new pool and add the filters to it. 136 * A pathname can satisfy any one of the specified filters. 137 * A split cannot have files from different pools. 138 */ 139 protected void createPool(PathFilter... filters) { 140 MultiPathFilter multi = new MultiPathFilter(); 141 for (PathFilter f: filters) { 142 multi.add(f); 143 } 144 pools.add(multi); 145 } 146 147 @Override 148 protected boolean isSplitable(JobContext context, Path file) { 149 final CompressionCodec codec = 150 new CompressionCodecFactory(context.getConfiguration()).getCodec(file); 151 if (null == codec) { 152 return true; 153 } 154 return codec instanceof SplittableCompressionCodec; 155 } 156 157 /** 158 * default constructor 159 */ 160 public CombineFileInputFormat() { 161 } 162 163 @Override 164 public List<InputSplit> getSplits(JobContext job) 165 throws IOException { 166 167 long minSizeNode = 0; 168 long minSizeRack = 0; 169 long maxSize = 0; 170 Configuration conf = job.getConfiguration(); 171 172 // the values specified by setxxxSplitSize() takes precedence over the 173 // values that might have been specified in the config 174 if (minSplitSizeNode != 0) { 175 minSizeNode = minSplitSizeNode; 176 } else { 177 minSizeNode = conf.getLong(SPLIT_MINSIZE_PERNODE, 0); 178 } 179 if (minSplitSizeRack != 0) { 180 minSizeRack = minSplitSizeRack; 181 } else { 182 minSizeRack = conf.getLong(SPLIT_MINSIZE_PERRACK, 0); 183 } 184 if (maxSplitSize != 0) { 185 maxSize = maxSplitSize; 186 } else { 187 maxSize = conf.getLong("mapreduce.input.fileinputformat.split.maxsize", 0); 188 } 189 if (minSizeNode != 0 && maxSize != 0 && minSizeNode > maxSize) { 190 throw new IOException("Minimum split size pernode " + minSizeNode + 191 " cannot be larger than maximum split size " + 192 maxSize); 193 } 194 if (minSizeRack != 0 && maxSize != 0 && minSizeRack > maxSize) { 195 throw new IOException("Minimum split size per rack" + minSizeRack + 196 " cannot be larger than maximum split size " + 197 maxSize); 198 } 199 if (minSizeRack != 0 && minSizeNode > minSizeRack) { 200 throw new IOException("Minimum split size per node" + minSizeNode + 201 " cannot be smaller than minimum split " + 202 "size per rack " + minSizeRack); 203 } 204 205 // all the files in input set 206 Path[] paths = FileUtil.stat2Paths( 207 listStatus(job).toArray(new FileStatus[0])); 208 List<InputSplit> splits = new ArrayList<InputSplit>(); 209 if (paths.length == 0) { 210 return splits; 211 } 212 213 // Convert them to Paths first. This is a costly operation and 214 // we should do it first, otherwise we will incur doing it multiple 215 // times, one time each for each pool in the next loop. 216 List<Path> newpaths = new LinkedList<Path>(); 217 for (int i = 0; i < paths.length; i++) { 218 FileSystem fs = paths[i].getFileSystem(conf); 219 Path p = fs.makeQualified(paths[i]); 220 newpaths.add(p); 221 } 222 paths = null; 223 224 // In one single iteration, process all the paths in a single pool. 225 // Processing one pool at a time ensures that a split contains paths 226 // from a single pool only. 227 for (MultiPathFilter onepool : pools) { 228 ArrayList<Path> myPaths = new ArrayList<Path>(); 229 230 // pick one input path. If it matches all the filters in a pool, 231 // add it to the output set 232 for (Iterator<Path> iter = newpaths.iterator(); iter.hasNext();) { 233 Path p = iter.next(); 234 if (onepool.accept(p)) { 235 myPaths.add(p); // add it to my output set 236 iter.remove(); 237 } 238 } 239 // create splits for all files in this pool. 240 getMoreSplits(job, myPaths.toArray(new Path[myPaths.size()]), 241 maxSize, minSizeNode, minSizeRack, splits); 242 } 243 244 // create splits for all files that are not in any pool. 245 getMoreSplits(job, newpaths.toArray(new Path[newpaths.size()]), 246 maxSize, minSizeNode, minSizeRack, splits); 247 248 // free up rackToNodes map 249 rackToNodes.clear(); 250 return splits; 251 } 252 253 /** 254 * Return all the splits in the specified set of paths 255 */ 256 private void getMoreSplits(JobContext job, Path[] paths, 257 long maxSize, long minSizeNode, long minSizeRack, 258 List<InputSplit> splits) 259 throws IOException { 260 Configuration conf = job.getConfiguration(); 261 262 // all blocks for all the files in input set 263 OneFileInfo[] files; 264 265 // mapping from a rack name to the list of blocks it has 266 HashMap<String, List<OneBlockInfo>> rackToBlocks = 267 new HashMap<String, List<OneBlockInfo>>(); 268 269 // mapping from a block to the nodes on which it has replicas 270 HashMap<OneBlockInfo, String[]> blockToNodes = 271 new HashMap<OneBlockInfo, String[]>(); 272 273 // mapping from a node to the list of blocks that it contains 274 HashMap<String, List<OneBlockInfo>> nodeToBlocks = 275 new HashMap<String, List<OneBlockInfo>>(); 276 277 files = new OneFileInfo[paths.length]; 278 if (paths.length == 0) { 279 return; 280 } 281 282 // populate all the blocks for all files 283 long totLength = 0; 284 for (int i = 0; i < paths.length; i++) { 285 files[i] = new OneFileInfo(paths[i], conf, isSplitable(job, paths[i]), 286 rackToBlocks, blockToNodes, nodeToBlocks, 287 rackToNodes, maxSize); 288 totLength += files[i].getLength(); 289 } 290 291 ArrayList<OneBlockInfo> validBlocks = new ArrayList<OneBlockInfo>(); 292 Set<String> nodes = new HashSet<String>(); 293 long curSplitSize = 0; 294 295 // process all nodes and create splits that are local 296 // to a node. 297 for (Iterator<Map.Entry<String, 298 List<OneBlockInfo>>> iter = nodeToBlocks.entrySet().iterator(); 299 iter.hasNext();) { 300 301 Map.Entry<String, List<OneBlockInfo>> one = iter.next(); 302 nodes.add(one.getKey()); 303 List<OneBlockInfo> blocksInNode = one.getValue(); 304 305 // for each block, copy it into validBlocks. Delete it from 306 // blockToNodes so that the same block does not appear in 307 // two different splits. 308 for (OneBlockInfo oneblock : blocksInNode) { 309 if (blockToNodes.containsKey(oneblock)) { 310 validBlocks.add(oneblock); 311 blockToNodes.remove(oneblock); 312 curSplitSize += oneblock.length; 313 314 // if the accumulated split size exceeds the maximum, then 315 // create this split. 316 if (maxSize != 0 && curSplitSize >= maxSize) { 317 // create an input split and add it to the splits array 318 addCreatedSplit(splits, nodes, validBlocks); 319 curSplitSize = 0; 320 validBlocks.clear(); 321 } 322 } 323 } 324 // if there were any blocks left over and their combined size is 325 // larger than minSplitNode, then combine them into one split. 326 // Otherwise add them back to the unprocessed pool. It is likely 327 // that they will be combined with other blocks from the 328 // same rack later on. 329 if (minSizeNode != 0 && curSplitSize >= minSizeNode) { 330 // create an input split and add it to the splits array 331 addCreatedSplit(splits, nodes, validBlocks); 332 } else { 333 for (OneBlockInfo oneblock : validBlocks) { 334 blockToNodes.put(oneblock, oneblock.hosts); 335 } 336 } 337 validBlocks.clear(); 338 nodes.clear(); 339 curSplitSize = 0; 340 } 341 342 // if blocks in a rack are below the specified minimum size, then keep them 343 // in 'overflow'. After the processing of all racks is complete, these 344 // overflow blocks will be combined into splits. 345 ArrayList<OneBlockInfo> overflowBlocks = new ArrayList<OneBlockInfo>(); 346 Set<String> racks = new HashSet<String>(); 347 348 // Process all racks over and over again until there is no more work to do. 349 while (blockToNodes.size() > 0) { 350 351 // Create one split for this rack before moving over to the next rack. 352 // Come back to this rack after creating a single split for each of the 353 // remaining racks. 354 // Process one rack location at a time, Combine all possible blocks that 355 // reside on this rack as one split. (constrained by minimum and maximum 356 // split size). 357 358 // iterate over all racks 359 for (Iterator<Map.Entry<String, List<OneBlockInfo>>> iter = 360 rackToBlocks.entrySet().iterator(); iter.hasNext();) { 361 362 Map.Entry<String, List<OneBlockInfo>> one = iter.next(); 363 racks.add(one.getKey()); 364 List<OneBlockInfo> blocks = one.getValue(); 365 366 // for each block, copy it into validBlocks. Delete it from 367 // blockToNodes so that the same block does not appear in 368 // two different splits. 369 boolean createdSplit = false; 370 for (OneBlockInfo oneblock : blocks) { 371 if (blockToNodes.containsKey(oneblock)) { 372 validBlocks.add(oneblock); 373 blockToNodes.remove(oneblock); 374 curSplitSize += oneblock.length; 375 376 // if the accumulated split size exceeds the maximum, then 377 // create this split. 378 if (maxSize != 0 && curSplitSize >= maxSize) { 379 // create an input split and add it to the splits array 380 addCreatedSplit(splits, getHosts(racks), validBlocks); 381 createdSplit = true; 382 break; 383 } 384 } 385 } 386 387 // if we created a split, then just go to the next rack 388 if (createdSplit) { 389 curSplitSize = 0; 390 validBlocks.clear(); 391 racks.clear(); 392 continue; 393 } 394 395 if (!validBlocks.isEmpty()) { 396 if (minSizeRack != 0 && curSplitSize >= minSizeRack) { 397 // if there is a minimum size specified, then create a single split 398 // otherwise, store these blocks into overflow data structure 399 addCreatedSplit(splits, getHosts(racks), validBlocks); 400 } else { 401 // There were a few blocks in this rack that 402 // remained to be processed. Keep them in 'overflow' block list. 403 // These will be combined later. 404 overflowBlocks.addAll(validBlocks); 405 } 406 } 407 curSplitSize = 0; 408 validBlocks.clear(); 409 racks.clear(); 410 } 411 } 412 413 assert blockToNodes.isEmpty(); 414 assert curSplitSize == 0; 415 assert validBlocks.isEmpty(); 416 assert racks.isEmpty(); 417 418 // Process all overflow blocks 419 for (OneBlockInfo oneblock : overflowBlocks) { 420 validBlocks.add(oneblock); 421 curSplitSize += oneblock.length; 422 423 // This might cause an exiting rack location to be re-added, 424 // but it should be ok. 425 for (int i = 0; i < oneblock.racks.length; i++) { 426 racks.add(oneblock.racks[i]); 427 } 428 429 // if the accumulated split size exceeds the maximum, then 430 // create this split. 431 if (maxSize != 0 && curSplitSize >= maxSize) { 432 // create an input split and add it to the splits array 433 addCreatedSplit(splits, getHosts(racks), validBlocks); 434 curSplitSize = 0; 435 validBlocks.clear(); 436 racks.clear(); 437 } 438 } 439 440 // Process any remaining blocks, if any. 441 if (!validBlocks.isEmpty()) { 442 addCreatedSplit(splits, getHosts(racks), validBlocks); 443 } 444 } 445 446 /** 447 * Create a single split from the list of blocks specified in validBlocks 448 * Add this new split into splitList. 449 */ 450 private void addCreatedSplit(List<InputSplit> splitList, 451 Collection<String> locations, 452 ArrayList<OneBlockInfo> validBlocks) { 453 // create an input split 454 Path[] fl = new Path[validBlocks.size()]; 455 long[] offset = new long[validBlocks.size()]; 456 long[] length = new long[validBlocks.size()]; 457 for (int i = 0; i < validBlocks.size(); i++) { 458 fl[i] = validBlocks.get(i).onepath; 459 offset[i] = validBlocks.get(i).offset; 460 length[i] = validBlocks.get(i).length; 461 } 462 463 // add this split to the list that is returned 464 CombineFileSplit thissplit = new CombineFileSplit(fl, offset, 465 length, locations.toArray(new String[0])); 466 splitList.add(thissplit); 467 } 468 469 /** 470 * This is not implemented yet. 471 */ 472 public abstract RecordReader<K, V> createRecordReader(InputSplit split, 473 TaskAttemptContext context) throws IOException; 474 475 /** 476 * information about one file from the File System 477 */ 478 private static class OneFileInfo { 479 private long fileSize; // size of the file 480 private OneBlockInfo[] blocks; // all blocks in this file 481 482 OneFileInfo(Path path, Configuration conf, 483 boolean isSplitable, 484 HashMap<String, List<OneBlockInfo>> rackToBlocks, 485 HashMap<OneBlockInfo, String[]> blockToNodes, 486 HashMap<String, List<OneBlockInfo>> nodeToBlocks, 487 HashMap<String, Set<String>> rackToNodes, 488 long maxSize) 489 throws IOException { 490 this.fileSize = 0; 491 492 // get block locations from file system 493 FileSystem fs = path.getFileSystem(conf); 494 FileStatus stat = fs.getFileStatus(path); 495 BlockLocation[] locations = fs.getFileBlockLocations(stat, 0, 496 stat.getLen()); 497 // create a list of all block and their locations 498 if (locations == null) { 499 blocks = new OneBlockInfo[0]; 500 } else { 501 if (!isSplitable) { 502 // if the file is not splitable, just create the one block with 503 // full file length 504 blocks = new OneBlockInfo[1]; 505 fileSize = stat.getLen(); 506 blocks[0] = new OneBlockInfo(path, 0, fileSize, locations[0] 507 .getHosts(), locations[0].getTopologyPaths()); 508 } else { 509 ArrayList<OneBlockInfo> blocksList = new ArrayList<OneBlockInfo>( 510 locations.length); 511 for (int i = 0; i < locations.length; i++) { 512 fileSize += locations[i].getLength(); 513 514 // each split can be a maximum of maxSize 515 long left = locations[i].getLength(); 516 long myOffset = locations[i].getOffset(); 517 long myLength = 0; 518 while (left > 0) { 519 if (maxSize == 0) { 520 myLength = left; 521 } else { 522 if (left > maxSize && left < 2 * maxSize) { 523 // if remainder is between max and 2*max - then 524 // instead of creating splits of size max, left-max we 525 // create splits of size left/2 and left/2. This is 526 // a heuristic to avoid creating really really small 527 // splits. 528 myLength = left / 2; 529 } else { 530 myLength = Math.min(maxSize, left); 531 } 532 } 533 OneBlockInfo oneblock = new OneBlockInfo(path, myOffset, 534 myLength, locations[i].getHosts(), locations[i] 535 .getTopologyPaths()); 536 left -= myLength; 537 myOffset += myLength; 538 539 blocksList.add(oneblock); 540 } 541 } 542 blocks = blocksList.toArray(new OneBlockInfo[blocksList.size()]); 543 } 544 545 for (OneBlockInfo oneblock : blocks) { 546 // add this block to the block --> node locations map 547 blockToNodes.put(oneblock, oneblock.hosts); 548 549 // For blocks that do not have host/rack information, 550 // assign to default rack. 551 String[] racks = null; 552 if (oneblock.hosts.length == 0) { 553 racks = new String[]{NetworkTopology.DEFAULT_RACK}; 554 } else { 555 racks = oneblock.racks; 556 } 557 558 // add this block to the rack --> block map 559 for (int j = 0; j < racks.length; j++) { 560 String rack = racks[j]; 561 List<OneBlockInfo> blklist = rackToBlocks.get(rack); 562 if (blklist == null) { 563 blklist = new ArrayList<OneBlockInfo>(); 564 rackToBlocks.put(rack, blklist); 565 } 566 blklist.add(oneblock); 567 if (!racks[j].equals(NetworkTopology.DEFAULT_RACK)) { 568 // Add this host to rackToNodes map 569 addHostToRack(rackToNodes, racks[j], oneblock.hosts[j]); 570 } 571 } 572 573 // add this block to the node --> block map 574 for (int j = 0; j < oneblock.hosts.length; j++) { 575 String node = oneblock.hosts[j]; 576 List<OneBlockInfo> blklist = nodeToBlocks.get(node); 577 if (blklist == null) { 578 blklist = new ArrayList<OneBlockInfo>(); 579 nodeToBlocks.put(node, blklist); 580 } 581 blklist.add(oneblock); 582 } 583 } 584 } 585 } 586 587 long getLength() { 588 return fileSize; 589 } 590 591 OneBlockInfo[] getBlocks() { 592 return blocks; 593 } 594 } 595 596 /** 597 * information about one block from the File System 598 */ 599 private static class OneBlockInfo { 600 Path onepath; // name of this file 601 long offset; // offset in file 602 long length; // length of this block 603 String[] hosts; // nodes on which this block resides 604 String[] racks; // network topology of hosts 605 606 OneBlockInfo(Path path, long offset, long len, 607 String[] hosts, String[] topologyPaths) { 608 this.onepath = path; 609 this.offset = offset; 610 this.hosts = hosts; 611 this.length = len; 612 assert (hosts.length == topologyPaths.length || 613 topologyPaths.length == 0); 614 615 // if the file system does not have any rack information, then 616 // use dummy rack location. 617 if (topologyPaths.length == 0) { 618 topologyPaths = new String[hosts.length]; 619 for (int i = 0; i < topologyPaths.length; i++) { 620 topologyPaths[i] = (new NodeBase(hosts[i], 621 NetworkTopology.DEFAULT_RACK)).toString(); 622 } 623 } 624 625 // The topology paths have the host name included as the last 626 // component. Strip it. 627 this.racks = new String[topologyPaths.length]; 628 for (int i = 0; i < topologyPaths.length; i++) { 629 this.racks[i] = (new NodeBase(topologyPaths[i])).getNetworkLocation(); 630 } 631 } 632 } 633 634 protected BlockLocation[] getFileBlockLocations( 635 FileSystem fs, FileStatus stat) throws IOException { 636 return fs.getFileBlockLocations(stat, 0, stat.getLen()); 637 } 638 639 private static void addHostToRack(HashMap<String, Set<String>> rackToNodes, 640 String rack, String host) { 641 Set<String> hosts = rackToNodes.get(rack); 642 if (hosts == null) { 643 hosts = new HashSet<String>(); 644 rackToNodes.put(rack, hosts); 645 } 646 hosts.add(host); 647 } 648 649 private Set<String> getHosts(Set<String> racks) { 650 Set<String> hosts = new HashSet<String>(); 651 for (String rack : racks) { 652 if (rackToNodes.containsKey(rack)) { 653 hosts.addAll(rackToNodes.get(rack)); 654 } 655 } 656 return hosts; 657 } 658 659 /** 660 * Accept a path only if any one of filters given in the 661 * constructor do. 662 */ 663 private static class MultiPathFilter implements PathFilter { 664 private List<PathFilter> filters; 665 666 public MultiPathFilter() { 667 this.filters = new ArrayList<PathFilter>(); 668 } 669 670 public MultiPathFilter(List<PathFilter> filters) { 671 this.filters = filters; 672 } 673 674 public void add(PathFilter one) { 675 filters.add(one); 676 } 677 678 public boolean accept(Path path) { 679 for (PathFilter filter : filters) { 680 if (filter.accept(path)) { 681 return true; 682 } 683 } 684 return false; 685 } 686 687 public String toString() { 688 StringBuffer buf = new StringBuffer(); 689 buf.append("["); 690 for (PathFilter f: filters) { 691 buf.append(f); 692 buf.append(","); 693 } 694 buf.append("]"); 695 return buf.toString(); 696 } 697 } 698 }