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