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.flow.stream.element; 023 024import java.util.concurrent.Callable; 025 026import cascading.CascadingException; 027import cascading.flow.FlowProcess; 028import cascading.flow.SliceCounters; 029import cascading.flow.StepCounters; 030import cascading.flow.stream.duct.Duct; 031import cascading.flow.stream.duct.DuctException; 032import cascading.tap.Tap; 033import cascading.tuple.TupleEntry; 034import cascading.tuple.TupleEntryIterator; 035import org.slf4j.Logger; 036import org.slf4j.LoggerFactory; 037 038/** 039 * 040 */ 041public class SourceStage extends ElementStage<Void, TupleEntry> implements Callable<Throwable>, InputSource 042 { 043 private static final Logger LOG = LoggerFactory.getLogger( SourceStage.class ); 044 045 private final Tap source; 046 047 public SourceStage( FlowProcess flowProcess, Tap source ) 048 { 049 super( flowProcess, source ); 050 this.source = source; 051 } 052 053 public Tap getSource() 054 { 055 return source; 056 } 057 058 @Override 059 public Throwable call() throws Exception 060 { 061 return map( null ); 062 } 063 064 @Override 065 public void run( Object input ) throws Throwable 066 { 067 Throwable throwable = map( input ); 068 069 if( throwable != null ) 070 throw throwable; 071 } 072 073 private Throwable map( Object input ) 074 { 075 Throwable localThrowable = null; 076 TupleEntryIterator iterator = null; 077 078 try 079 { 080 next.start( this ); 081 082 // input may be null 083 iterator = source.openForRead( flowProcess, input ); 084 085 while( iterator.hasNext() ) 086 { 087 if( Thread.interrupted() ) 088 throw new InterruptedException( "thread interrupted" ); 089 090 TupleEntry tupleEntry; 091 092 try 093 { 094 tupleEntry = iterator.next(); 095 flowProcess.increment( StepCounters.Tuples_Read, 1 ); 096 flowProcess.increment( SliceCounters.Tuples_Read, 1 ); 097 } 098 catch( OutOfMemoryError error ) 099 { 100 handleReThrowableException( "out of memory, try increasing task memory allocation", error ); 101 continue; 102 } 103 catch( CascadingException exception ) 104 { 105 handleException( exception, null ); 106 continue; 107 } 108 catch( Throwable throwable ) 109 { 110 handleException( new DuctException( "internal error", throwable ), null ); 111 continue; 112 } 113 114 next.receive( this, 0, tupleEntry ); 115 } 116 117 next.complete( this ); 118 } 119 catch( InterruptedException exception ) 120 { 121 // do nothing -- let finally run 122 } 123 catch( Throwable throwable ) 124 { 125 if( !( throwable instanceof OutOfMemoryError ) ) 126 LOG.error( "caught throwable", throwable ); 127 128 return throwable; 129 } 130 finally 131 { 132 try 133 { 134 if( iterator != null ) 135 iterator.close(); 136 } 137 catch( Throwable currentThrowable ) 138 { 139 if( !( currentThrowable instanceof OutOfMemoryError ) ) 140 LOG.warn( "failed closing iterator", currentThrowable ); 141 142 localThrowable = currentThrowable; 143 } 144 } 145 146 return localThrowable; 147 } 148 149 @Override 150 public void initialize() 151 { 152 } 153 154 @Override 155 public void receive( Duct previous, int ordinal, Void nada ) 156 { 157 throw new UnsupportedOperationException( "use call() instead" ); 158 } 159 }