001/* 002 * Copyright (c) 2007-2022 The Cascading Authors. All Rights Reserved. 003 * 004 * Project and contact information: https://cascading.wensel.net/ 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 021package cascading.tap.hadoop; 022 023import java.beans.ConstructorProperties; 024import java.io.IOException; 025import java.util.ArrayList; 026import java.util.List; 027 028import cascading.flow.FlowProcess; 029import cascading.scheme.Scheme; 030import cascading.tap.MultiSourceTap; 031import cascading.tap.TapException; 032import cascading.util.LazyIterable; 033import org.apache.hadoop.conf.Configuration; 034import org.apache.hadoop.fs.FileStatus; 035import org.apache.hadoop.fs.FileSystem; 036import org.apache.hadoop.fs.Path; 037import org.apache.hadoop.fs.PathFilter; 038import org.apache.hadoop.mapred.JobConf; 039import org.apache.hadoop.mapred.RecordReader; 040 041/** 042 * Class GlobHfs is a type of {@link cascading.tap.MultiSourceTap} that accepts Hadoop style 'file globing' expressions so 043 * multiple files that match the given pattern may be used as the input sources for a given {@link cascading.flow.Flow}. 044 * <p> 045 * See {@link FileSystem#globStatus(org.apache.hadoop.fs.Path)} for details on the globing syntax. But in short 046 * it is similar to standard regular expressions except alternation is done via {foo,bar} instead of (foo|bar). 047 * <p> 048 * Note that a {@link cascading.flow.Flow} sourcing from GlobHfs is not currently compatible with the {@link cascading.cascade.Cascade} 049 * scheduler. GlobHfs expects the files and paths to exist so the wildcards can be resolved into concrete values so 050 * that the scheduler can order the Flows properly. 051 * <p> 052 * Note that globing can match files or directories. It may consume less resources to match directories and let 053 * Hadoop include all sub-files immediately contained in the directory instead of enumerating every individual file. 054 * Ending the glob path with a {@code /} should match only directories. 055 * 056 * @see Hfs 057 * @see cascading.tap.MultiSourceTap 058 * @see FileSystem 059 */ 060public class GlobHfs extends MultiSourceTap<Hfs, Configuration, RecordReader> 061 { 062 /** Field pathPattern */ 063 private final String pathPattern; 064 /** Field pathFilter */ 065 private final PathFilter pathFilter; 066 067 /** 068 * Constructor GlobHfs creates a new GlobHfs instance. 069 * 070 * @param scheme of type Scheme 071 * @param pathPattern of type String 072 */ 073 @ConstructorProperties({"scheme", "pathPattern"}) 074 public GlobHfs( Scheme<Configuration, RecordReader, ?, ?, ?> scheme, String pathPattern ) 075 { 076 this( scheme, pathPattern, null ); 077 } 078 079 /** 080 * Constructor GlobHfs creates a new GlobHfs instance. 081 * 082 * @param scheme of type Scheme 083 * @param pathPattern of type String 084 * @param pathFilter of type PathFilter 085 */ 086 @ConstructorProperties({"scheme", "pathPattern", "pathFilter"}) 087 public GlobHfs( Scheme<Configuration, RecordReader, ?, ?, ?> scheme, String pathPattern, PathFilter pathFilter ) 088 { 089 super( scheme ); 090 this.pathPattern = pathPattern; 091 this.pathFilter = pathFilter; 092 } 093 094 @Override 095 public String getIdentifier() 096 { 097 return pathPattern; 098 } 099 100 @Override 101 protected Hfs[] getTaps() 102 { 103 return initTapsInternal( new JobConf() ); 104 } 105 106 private Hfs[] initTapsInternal( Configuration conf ) 107 { 108 if( taps != null ) 109 return taps; 110 111 try 112 { 113 taps = makeTaps( conf ); 114 } 115 catch( IOException exception ) 116 { 117 throw new TapException( "unable to resolve taps for globing path: " + pathPattern ); 118 } 119 120 return taps; 121 } 122 123 private Hfs[] makeTaps( Configuration conf ) throws IOException 124 { 125 FileStatus[] statusList; 126 127 Path path = new Path( pathPattern ); 128 129 FileSystem fileSystem = path.getFileSystem( conf ); 130 131 if( pathFilter == null ) 132 statusList = fileSystem.globStatus( path ); 133 else 134 statusList = fileSystem.globStatus( path, pathFilter ); 135 136 if( statusList == null || statusList.length == 0 ) 137 throw new TapException( "unable to find paths matching path pattern: " + pathPattern ); 138 139 List<Hfs> notEmpty = new ArrayList<Hfs>(); 140 141 for( int i = 0; i < statusList.length; i++ ) 142 { 143 // remove empty files. some hadoop versions return non-zero for dirs 144 // so this jives with the expectations set in the above javadoc 145 if( statusList[ i ].isDir() || statusList[ i ].getLen() != 0 ) 146 notEmpty.add( new Hfs( getScheme(), statusList[ i ].getPath().toString() ) ); 147 } 148 149 if( notEmpty.isEmpty() ) 150 throw new TapException( "all paths matching path pattern are zero length and not directories: " + pathPattern ); 151 152 return notEmpty.toArray( new Hfs[ notEmpty.size() ] ); 153 } 154 155 @Override 156 public void sourceConfInit( FlowProcess<? extends Configuration> process, Configuration conf ) 157 { 158 Hfs[] taps = initTapsInternal( conf ); 159 160 taps[ 0 ].sourceConfInitAddInputPaths( conf, new LazyIterable<Hfs, Path>( taps ) 161 { 162 @Override 163 protected Path convert( Hfs next ) 164 { 165 return next.getPath(); // we are building fully qualified paths above 166 } 167 } ); 168 169 taps[ 0 ].sourceConfInitComplete( process, conf ); 170 } 171 172 @Override 173 public boolean equals( Object object ) 174 { 175 if( this == object ) 176 return true; 177 if( object == null || getClass() != object.getClass() ) 178 return false; 179 180 GlobHfs globHfs = (GlobHfs) object; 181 182 // do not compare tap arrays, these values should be sufficient to show identity 183 if( getScheme() != null ? !getScheme().equals( globHfs.getScheme() ) : globHfs.getScheme() != null ) 184 return false; 185 if( pathFilter != null ? !pathFilter.equals( globHfs.pathFilter ) : globHfs.pathFilter != null ) 186 return false; 187 if( pathPattern != null ? !pathPattern.equals( globHfs.pathPattern ) : globHfs.pathPattern != null ) 188 return false; 189 190 return true; 191 } 192 193 @Override 194 public int hashCode() 195 { 196 int result = pathPattern != null ? pathPattern.hashCode() : 0; 197 result = 31 * result + ( pathFilter != null ? pathFilter.hashCode() : 0 ); 198 return result; 199 } 200 201 @Override 202 public String toString() 203 { 204 return "GlobHfs[" + pathPattern + ']'; 205 } 206 }