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; 023 024import java.beans.ConstructorProperties; 025import java.io.IOException; 026import java.util.ArrayList; 027import java.util.Arrays; 028import java.util.HashSet; 029import java.util.Iterator; 030import java.util.List; 031import java.util.Map; 032import java.util.Set; 033 034import cascading.flow.FlowProcess; 035import cascading.scheme.NullScheme; 036import cascading.scheme.Scheme; 037import cascading.tuple.Fields; 038import cascading.tuple.TupleEntry; 039import cascading.tuple.TupleEntryCollector; 040import cascading.util.Util; 041import org.slf4j.Logger; 042import org.slf4j.LoggerFactory; 043 044/** 045 * Class MultiSinkTap is both a {@link cascading.tap.CompositeTap} and {@link cascading.tap.SinkTap} that can write to 046 * multiple child {@link cascading.tap.Tap} instances simultaneously. 047 * <p> 048 * It is the counterpart to {@link cascading.tap.MultiSourceTap}. 049 * <p> 050 * Note all child Tap instances may or may not have the same declared Fields. In the case they do not, all 051 * sink fields will be merged into a single Fields instance via {@link Fields#merge(cascading.tuple.Fields...)}. 052 */ 053public class MultiSinkTap<Child extends Tap, Config, Output> extends SinkTap<Config, Output> implements CompositeTap<Child> 054 { 055 /** Field LOG */ 056 private static final Logger LOG = LoggerFactory.getLogger( MultiSinkTap.class ); 057 058 /** Field taps */ 059 private final Child[] taps; 060 /** Field tempPath */ 061 private final String tempPath = "__multisink_placeholder_" + Util.createUniqueID(); 062 /** Field childConfigs */ 063 private List<Map<String, String>> childConfigs; 064 065 private class MultiSinkCollector extends TupleEntryCollector 066 { 067 TupleEntryCollector[] collectors; 068 069 public <C extends Config> MultiSinkCollector( FlowProcess<C> flowProcess, Tap<Config, ?, ?>... taps ) throws IOException 070 { 071 super( Fields.asDeclaration( getSinkFields() ) ); 072 073 collectors = new TupleEntryCollector[ taps.length ]; 074 075 C conf = flowProcess.getConfigCopy(); 076 077 for( int i = 0; i < taps.length; i++ ) 078 { 079 C mergedConf = childConfigs == null ? conf : flowProcess.mergeMapIntoConfig( conf, childConfigs.get( i ) ); 080 Tap<Config, ?, ?> tap = taps[ i ]; 081 LOG.info( "opening for write: {}", tap.toString() ); 082 083 collectors[ i ] = tap.openForWrite( flowProcess.copyWith( mergedConf ), null ); 084 } 085 } 086 087 protected void collect( TupleEntry tupleEntry ) throws IOException 088 { 089 for( int i = 0; i < taps.length; i++ ) 090 collectors[ i ].add( tupleEntry ); 091 } 092 093 @Override 094 public void close() 095 { 096 super.close(); 097 098 try 099 { 100 for( TupleEntryCollector collector : collectors ) 101 { 102 try 103 { 104 collector.close(); 105 } 106 catch( Exception exception ) 107 { 108 LOG.warn( "exception closing TupleEntryCollector", exception ); 109 } 110 } 111 } 112 finally 113 { 114 collectors = null; 115 } 116 } 117 } 118 119 /** 120 * Constructor MultiSinkTap creates a new MultiSinkTap instance. 121 * 122 * @param taps of type Tap... 123 */ 124 @ConstructorProperties({"taps"}) 125 public MultiSinkTap( Child... taps ) 126 { 127 this.taps = taps; 128 } 129 130 protected Child[] getTaps() 131 { 132 return taps; 133 } 134 135 @Override 136 public Iterator<Child> getChildTaps() 137 { 138 return Arrays.asList( getTaps() ).iterator(); 139 } 140 141 @Override 142 public long getNumChildTaps() 143 { 144 return getTaps().length; 145 } 146 147 @Override 148 public String getIdentifier() 149 { 150 return tempPath; 151 } 152 153 @Override 154 public void presentSinkFields( FlowProcess<? extends Config> flowProcess, Fields fields ) 155 { 156 for( Tap child : getTaps() ) 157 child.presentSinkFields( flowProcess, fields ); 158 } 159 160 @Override 161 public TupleEntryCollector openForWrite( FlowProcess<? extends Config> flowProcess, Output output ) throws IOException 162 { 163 return new MultiSinkCollector( flowProcess, getTaps() ); 164 } 165 166 @Override 167 public void sinkConfInit( FlowProcess<? extends Config> flowProcess, Config conf ) 168 { 169 bridge( flowProcess, conf ); 170 } 171 172 private void bridge( FlowProcess flowProcess, Object conf ) 173 { 174 childConfigs = new ArrayList<>(); 175 176 for( int i = 0; i < getTaps().length; i++ ) 177 { 178 Tap tap = getTaps()[ i ]; 179 Object newConfig = flowProcess.copyConfig( conf ); 180 181 tap.sinkConfInit( flowProcess, newConfig ); 182 183 childConfigs.add( flowProcess.diffConfigIntoMap( conf, newConfig ) ); 184 } 185 } 186 187 @Override 188 public boolean createResource( Config conf ) throws IOException 189 { 190 for( Tap tap : getTaps() ) 191 { 192 if( !tap.createResource( conf ) ) 193 return false; 194 } 195 196 return true; 197 } 198 199 @Override 200 public boolean deleteResource( Config conf ) throws IOException 201 { 202 for( Tap tap : getTaps() ) 203 { 204 if( !tap.deleteResource( conf ) ) 205 return false; 206 } 207 208 return true; 209 } 210 211 @Override 212 public boolean commitResource( Config conf ) throws IOException 213 { 214 for( Tap tap : getTaps() ) 215 { 216 if( !tap.commitResource( conf ) ) 217 return false; 218 } 219 220 return true; 221 } 222 223 @Override 224 public boolean rollbackResource( Config conf ) throws IOException 225 { 226 for( Tap tap : getTaps() ) 227 { 228 if( !tap.rollbackResource( conf ) ) 229 return false; 230 } 231 232 return true; 233 } 234 235 @Override 236 public boolean resourceExists( Config conf ) throws IOException 237 { 238 for( Tap tap : getTaps() ) 239 { 240 if( !tap.resourceExists( conf ) ) 241 return false; 242 } 243 244 return true; 245 } 246 247 @Override 248 public long getModifiedTime( Config conf ) throws IOException 249 { 250 long modified = getTaps()[ 0 ].getModifiedTime( conf ); 251 252 for( int i = 1; i < getTaps().length; i++ ) 253 modified = Math.max( getTaps()[ i ].getModifiedTime( conf ), modified ); 254 255 return modified; 256 } 257 258 @Override 259 public Scheme getScheme() 260 { 261 if( super.getScheme() != null ) 262 return super.getScheme(); 263 264 Set<Fields> fields = new HashSet<Fields>(); 265 266 for( Tap child : getTaps() ) 267 fields.add( child.getSinkFields() ); 268 269 // if all schemes have the same sink fields, the just use the scheme 270 if( fields.size() == 1 ) 271 { 272 setScheme( getTaps()[ 0 ].getScheme() ); 273 return super.getScheme(); 274 } 275 276 Fields allFields = Fields.merge( fields.toArray( new Fields[ fields.size() ] ) ); 277 278 setScheme( new NullScheme( allFields, allFields ) ); 279 280 return super.getScheme(); 281 } 282 283 @Override 284 public String toString() 285 { 286 return "MultiSinkTap[" + ( taps == null ? "none" : Arrays.asList( taps ) ) + ']'; 287 } 288 289 @Override 290 public boolean equals( Object o ) 291 { 292 if( this == o ) 293 return true; 294 if( !( o instanceof MultiSinkTap ) ) 295 return false; 296 if( !super.equals( o ) ) 297 return false; 298 299 MultiSinkTap that = (MultiSinkTap) o; 300 301 if( !Arrays.equals( taps, that.taps ) ) 302 return false; 303 304 return true; 305 } 306 307 @Override 308 public int hashCode() 309 { 310 int result = super.hashCode(); 311 result = 31 * result + ( taps != null ? Arrays.hashCode( taps ) : 0 ); 312 return result; 313 } 314 }