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.scheme.local; 023 024import java.beans.ConstructorProperties; 025import java.io.IOException; 026import java.io.InputStream; 027import java.io.InputStreamReader; 028import java.io.LineNumberReader; 029import java.io.OutputStream; 030import java.io.OutputStreamWriter; 031import java.io.PrintWriter; 032import java.io.UnsupportedEncodingException; 033import java.nio.charset.Charset; 034import java.util.Properties; 035 036import cascading.flow.FlowProcess; 037import cascading.management.annotation.Property; 038import cascading.management.annotation.PropertyDescription; 039import cascading.management.annotation.Visibility; 040import cascading.scheme.FileFormat; 041import cascading.scheme.SinkCall; 042import cascading.scheme.SourceCall; 043import cascading.tap.Tap; 044import cascading.tap.TapException; 045import cascading.tuple.Fields; 046import cascading.tuple.TupleEntry; 047 048/** 049 * A TextLine is a type of {@link cascading.scheme.Scheme} for plain text files. Files are broken into 050 * lines. Either line-feed or carriage-return are used to signal end of line. 051 * <p> 052 * By default, this scheme returns a {@link cascading.tuple.Tuple} with two fields, "num" and "line". Where "num" 053 * is the line number for "line". 054 * <p> 055 * Many of the constructors take both "sourceFields" and "sinkFields". sourceFields denote the field names 056 * to be used instead of the names "num" and "line". sinkFields is a selector and is by default {@link Fields#ALL}. 057 * Any available field names can be given if only a subset of the incoming fields should be used. 058 * <p> 059 * If a {@link Fields} instance is passed on the constructor as sourceFields having only one field, the return tuples 060 * will simply be the "line" value using the given field name. 061 * <p> 062 * Note that TextLine will concatenate all the Tuple values for the selected fields with a TAB delimiter before 063 * writing out the line. 064 * <p> 065 * By default, all text is encoded/decoded as UTF-8. This can be changed via the {@code charsetName} constructor 066 * argument. 067 * <p> 068 * In order to read or write a compressed files, pass a {@link cascading.scheme.local.CompressorScheme.Compressor} 069 * instance to the appropriate constructors. See {@link Compressors} for provided compression algorithms. 070 * 071 * @see TextDelimited 072 * @see Compressors 073 */ 074public class TextLine extends CompressorScheme<LineNumberReader, PrintWriter> implements FileFormat 075 { 076 public static final String DEFAULT_CHARSET = "UTF-8"; 077 public static final Fields DEFAULT_SOURCE_FIELDS = new Fields( "num", "line" ).applyTypes( Integer.TYPE, String.class ); 078 079 private String charsetName = DEFAULT_CHARSET; 080 081 /** 082 * Creates a new TextLine instance that sources "num" and "line" fields, and sinks all incoming fields, where 083 * "num" is the line number of the line in the input file. 084 */ 085 public TextLine() 086 { 087 super( DEFAULT_SOURCE_FIELDS, Fields.ALL ); 088 } 089 090 /** 091 * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the 092 * subsequent tuples. 093 * 094 * @param sourceFields of Fields 095 */ 096 @ConstructorProperties({"sourceFields"}) 097 public TextLine( Fields sourceFields ) 098 { 099 super( sourceFields ); 100 101 verify( sourceFields ); 102 } 103 104 /** 105 * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the 106 * subsequent tuples. 107 * 108 * @param sourceFields of Fields 109 * @param charsetName of type String 110 */ 111 @ConstructorProperties({"sourceFields", "charsetName"}) 112 public TextLine( Fields sourceFields, String charsetName ) 113 { 114 super( sourceFields ); 115 116 // throws an exception if not found 117 setCharsetName( charsetName ); 118 119 verify( sourceFields ); 120 } 121 122 /** 123 * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the 124 * subsequent tuples. 125 * 126 * @param sourceFields of Fields 127 * @param sinkFields of Fields 128 */ 129 @ConstructorProperties({"sourceFields", "sinkFields"}) 130 public TextLine( Fields sourceFields, Fields sinkFields ) 131 { 132 super( sourceFields, sinkFields ); 133 134 verify( sourceFields ); 135 } 136 137 /** 138 * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the 139 * subsequent tuples. 140 * 141 * @param sourceFields of Fields 142 * @param sinkFields of Fields 143 * @param charsetName of type String 144 */ 145 @ConstructorProperties({"sourceFields", "sinkFields", "charsetName"}) 146 public TextLine( Fields sourceFields, Fields sinkFields, String charsetName ) 147 { 148 super( sourceFields, sinkFields ); 149 150 // throws an exception if not found 151 setCharsetName( charsetName ); 152 153 verify( sourceFields ); 154 } 155 156 /** 157 * Creates a new TextLine instance that sources "num" and "line" fields, and sinks all incoming fields, where 158 * "num" is the line number of the line in the input file. 159 */ 160 public TextLine( Compressor compressor ) 161 { 162 super( DEFAULT_SOURCE_FIELDS, Fields.ALL, compressor ); 163 } 164 165 /** 166 * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the 167 * subsequent tuples. 168 * 169 * @param sourceFields of Fields 170 * @param compressor of type Compressor 171 */ 172 @ConstructorProperties({"sourceFields", "compressor"}) 173 public TextLine( Fields sourceFields, Compressor compressor ) 174 { 175 super( sourceFields, compressor ); 176 177 verify( sourceFields ); 178 } 179 180 /** 181 * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the 182 * subsequent tuples. 183 * 184 * @param sourceFields of Fields 185 * @param compressor of type Compressor 186 * @param charsetName of type String 187 */ 188 @ConstructorProperties({"sourceFields", "compressor", "charsetName"}) 189 public TextLine( Fields sourceFields, Compressor compressor, String charsetName ) 190 { 191 super( sourceFields, compressor ); 192 193 // throws an exception if not found 194 setCharsetName( charsetName ); 195 196 verify( sourceFields ); 197 } 198 199 /** 200 * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the 201 * subsequent tuples. 202 * 203 * @param sourceFields of Fields 204 * @param sinkFields of Fields 205 * @param compressor of type Compressor 206 */ 207 @ConstructorProperties({"sourceFields", "sinkFields", "compressor"}) 208 public TextLine( Fields sourceFields, Fields sinkFields, Compressor compressor ) 209 { 210 super( sourceFields, sinkFields, compressor ); 211 212 verify( sourceFields ); 213 } 214 215 /** 216 * Creates a new TextLine instance. If sourceFields has one field, only the text line will be returned in the 217 * subsequent tuples. 218 * 219 * @param sourceFields of Fields 220 * @param sinkFields of Fields 221 * @param compressor of type Compressor 222 * @param charsetName of type String 223 */ 224 @ConstructorProperties({"sourceFields", "sinkFields", "compressor", "charsetName"}) 225 public TextLine( Fields sourceFields, Fields sinkFields, Compressor compressor, String charsetName ) 226 { 227 super( sourceFields, sinkFields, compressor ); 228 229 // throws an exception if not found 230 setCharsetName( charsetName ); 231 232 verify( sourceFields ); 233 } 234 235 protected void setCharsetName( String charsetName ) 236 { 237 if( charsetName != null ) 238 this.charsetName = charsetName; 239 240 Charset.forName( this.charsetName ); 241 } 242 243 @Property(name = "charset", visibility = Visibility.PUBLIC) 244 @PropertyDescription("character set used.") 245 public String getCharsetName() 246 { 247 return charsetName; 248 } 249 250 protected void verify( Fields sourceFields ) 251 { 252 if( sourceFields.size() < 1 || sourceFields.size() > 2 ) 253 throw new IllegalArgumentException( "this scheme requires either one or two source fields, given [" + sourceFields + "]" ); 254 } 255 256 public LineNumberReader createInput( InputStream inputStream ) 257 { 258 try 259 { 260 return new LineNumberReader( new InputStreamReader( inputStream, charsetName ) ); 261 } 262 catch( UnsupportedEncodingException exception ) 263 { 264 throw new TapException( exception ); 265 } 266 } 267 268 public PrintWriter createOutput( OutputStream outputStream ) 269 { 270 try 271 { 272 return new PrintWriter( new OutputStreamWriter( outputStream, charsetName ) ); 273 } 274 catch( UnsupportedEncodingException exception ) 275 { 276 throw new TapException( exception ); 277 } 278 } 279 280 @Override 281 public void presentSourceFields( FlowProcess<? extends Properties> process, Tap tap, Fields fields ) 282 { 283 // do nothing 284 } 285 286 @Override 287 public void presentSinkFields( FlowProcess<? extends Properties> process, Tap tap, Fields fields ) 288 { 289 // do nothing 290 } 291 292 @Override 293 public void sourceConfInit( FlowProcess<? extends Properties> flowProcess, Tap<Properties, InputStream, OutputStream> tap, Properties conf ) 294 { 295 } 296 297 @Override 298 public void sinkConfInit( FlowProcess<? extends Properties> flowProcess, Tap<Properties, InputStream, OutputStream> tap, Properties conf ) 299 { 300 } 301 302 @Override 303 public void sourcePrepare( FlowProcess<? extends Properties> flowProcess, SourceCall<LineNumberReader, InputStream> sourceCall ) throws IOException 304 { 305 sourceCall.setContext( createInput( sourceCall.getInput() ) ); 306 } 307 308 @Override 309 public void sourceRePrepare( FlowProcess<? extends Properties> flowProcess, SourceCall<LineNumberReader, InputStream> sourceCall ) throws IOException 310 { 311 sourceCall.setContext( createInput( sourceCall.getInput() ) ); 312 } 313 314 @Override 315 public boolean source( FlowProcess<? extends Properties> flowProcess, SourceCall<LineNumberReader, InputStream> sourceCall ) throws IOException 316 { 317 // first line is 0, this matches offset being zero, so when throwing out the first line for comments 318 int lineNumber = sourceCall.getContext().getLineNumber(); 319 String line = sourceCall.getContext().readLine(); 320 321 if( line == null ) 322 return false; 323 324 TupleEntry incomingEntry = sourceCall.getIncomingEntry(); 325 326 if( getSourceFields().size() == 1 ) 327 { 328 incomingEntry.setObject( 0, line ); 329 } 330 else 331 { 332 incomingEntry.setInteger( 0, lineNumber ); 333 incomingEntry.setString( 1, line ); 334 } 335 336 return true; 337 } 338 339 @Override 340 public void sourceCleanup( FlowProcess<? extends Properties> flowProcess, SourceCall<LineNumberReader, InputStream> sourceCall ) throws IOException 341 { 342 sourceCall.setContext( null ); 343 } 344 345 @Override 346 public void sinkPrepare( FlowProcess<? extends Properties> flowProcess, SinkCall<PrintWriter, OutputStream> sinkCall ) throws IOException 347 { 348 sinkCall.setContext( createOutput( sinkCall.getOutput() ) ); 349 } 350 351 @Override 352 public void sink( FlowProcess<? extends Properties> flowProcess, SinkCall<PrintWriter, OutputStream> sinkCall ) throws IOException 353 { 354 sinkCall.getContext().println( sinkCall.getOutgoingEntry().getTuple().toString() ); 355 } 356 357 @Override 358 public void sinkCleanup( FlowProcess<? extends Properties> flowProcess, SinkCall<PrintWriter, OutputStream> sinkCall ) throws IOException 359 { 360 sinkCall.getContext().flush(); 361 sinkCall.setContext( null ); 362 } 363 364 @Override 365 public String getExtension() 366 { 367 return "txt"; 368 } 369 }