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