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.scheme.hadoop; 022 023import java.beans.ConstructorProperties; 024import java.io.IOException; 025 026import cascading.flow.FlowProcess; 027import cascading.scheme.FileFormat; 028import cascading.scheme.Scheme; 029import cascading.scheme.SinkCall; 030import cascading.scheme.SourceCall; 031import cascading.tap.Tap; 032import cascading.tuple.Fields; 033import cascading.tuple.Tuple; 034import cascading.tuple.TupleEntry; 035import org.apache.hadoop.conf.Configuration; 036import org.apache.hadoop.mapred.InputFormat; 037import org.apache.hadoop.mapred.OutputCollector; 038import org.apache.hadoop.mapred.OutputFormat; 039import org.apache.hadoop.mapred.RecordReader; 040import org.apache.hadoop.mapred.SequenceFileInputFormat; 041import org.apache.hadoop.mapred.SequenceFileOutputFormat; 042 043/** 044 * A SequenceFile is a type of {@link cascading.scheme.Scheme}, which is a flat file consisting of 045 * binary key/value pairs. This is a space and time efficient means to store data. 046 */ 047public class SequenceFile extends Scheme<Configuration, RecordReader, OutputCollector, Object[], Void> implements FileFormat 048 { 049 /** Protected for use by TempDfs and other subclasses. Not for general consumption. */ 050 protected SequenceFile() 051 { 052 super( null ); 053 } 054 055 /** 056 * Creates a new SequenceFile instance that stores the given field names. 057 * 058 * @param fields 059 */ 060 @ConstructorProperties({"fields"}) 061 public SequenceFile( Fields fields ) 062 { 063 super( fields, fields ); 064 } 065 066 @Override 067 public void sourceConfInit( FlowProcess<? extends Configuration> flowProcess, Tap<Configuration, RecordReader, OutputCollector> tap, Configuration conf ) 068 { 069 conf.setBoolean( "mapred.mapper.new-api", false ); 070 conf.setClass( "mapred.input.format.class", SequenceFileInputFormat.class, InputFormat.class ); 071 } 072 073 @Override 074 public void sinkConfInit( FlowProcess<? extends Configuration> flowProcess, Tap<Configuration, RecordReader, OutputCollector> tap, Configuration conf ) 075 { 076 conf.setBoolean( "mapred.mapper.new-api", false ); 077 conf.setClass( "mapred.output.key.class", Tuple.class, Object.class ); 078 conf.setClass( "mapred.output.value.class", Tuple.class, Object.class ); 079 conf.setClass( "mapred.output.format.class", SequenceFileOutputFormat.class, OutputFormat.class ); 080 } 081 082 @Override 083 public void sourcePrepare( FlowProcess<? extends Configuration> flowProcess, SourceCall<Object[], RecordReader> sourceCall ) 084 { 085 Object[] pair = new Object[]{ 086 sourceCall.getInput().createKey(), 087 sourceCall.getInput().createValue() 088 }; 089 090 sourceCall.setContext( pair ); 091 } 092 093 @Override 094 public boolean source( FlowProcess<? extends Configuration> flowProcess, SourceCall<Object[], RecordReader> sourceCall ) throws IOException 095 { 096 Tuple key = (Tuple) sourceCall.getContext()[ 0 ]; 097 Tuple value = (Tuple) sourceCall.getContext()[ 1 ]; 098 boolean result = sourceCall.getInput().next( key, value ); 099 100 if( !result ) 101 return false; 102 103 TupleEntry entry = sourceCall.getIncomingEntry(); 104 105 if( entry.hasTypes() ) 106 entry.setCanonicalTuple( value ); 107 else 108 entry.setTuple( value ); 109 110 return true; 111 } 112 113 @Override 114 public void sourceCleanup( FlowProcess<? extends Configuration> flowProcess, SourceCall<Object[], RecordReader> sourceCall ) 115 { 116 sourceCall.setContext( null ); 117 } 118 119 @Override 120 public void sink( FlowProcess<? extends Configuration> flowProcess, SinkCall<Void, OutputCollector> sinkCall ) throws IOException 121 { 122 sinkCall.getOutput().collect( Tuple.NULL, sinkCall.getOutgoingEntry().getTuple() ); 123 } 124 125 @Override 126 public String getExtension() 127 { 128 return "seq"; 129 } 130 }