001    /**
002     * Licensed to the Apache Software Foundation (ASF) under one
003     * or more contributor license agreements.  See the NOTICE file
004     * distributed with this work for additional information
005     * regarding copyright ownership.  The ASF licenses this file
006     * to you under the Apache License, Version 2.0 (the
007     * "License"); you may not use this file except in compliance
008     * with the License.  You may obtain a copy of the License at
009     *
010     *     http://www.apache.org/licenses/LICENSE-2.0
011     *
012     * Unless required by applicable law or agreed to in writing, software
013     * distributed under the License is distributed on an "AS IS" BASIS,
014     * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015     * See the License for the specific language governing permissions and
016     * limitations under the License.
017     */
018    
019    package org.apache.hadoop.mapred.lib.aggregate;
020    
021    import java.io.IOException;
022    import java.util.Iterator;
023    
024    import org.apache.hadoop.classification.InterfaceAudience;
025    import org.apache.hadoop.classification.InterfaceStability;
026    import org.apache.hadoop.io.Text;
027    import org.apache.hadoop.io.Writable;
028    import org.apache.hadoop.io.WritableComparable;
029    import org.apache.hadoop.mapred.JobConf;
030    import org.apache.hadoop.mapred.OutputCollector;
031    import org.apache.hadoop.mapred.Reporter;
032    
033    /**
034     * This class implements the generic combiner of Aggregate.
035     */
036    @InterfaceAudience.Public
037    @InterfaceStability.Stable
038    public class ValueAggregatorCombiner<K1 extends WritableComparable,
039                                         V1 extends Writable>
040      extends ValueAggregatorJobBase<K1, V1> {
041    
042      /**
043       * Combiner does not need to configure.
044       */
045      public void configure(JobConf job) {
046    
047      }
048    
049      /** Combines values for a given key.  
050       * @param key the key is expected to be a Text object, whose prefix indicates
051       * the type of aggregation to aggregate the values. 
052       * @param values the values to combine
053       * @param output to collect combined values
054       */
055      public void reduce(Text key, Iterator<Text> values,
056                         OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
057        String keyStr = key.toString();
058        int pos = keyStr.indexOf(ValueAggregatorDescriptor.TYPE_SEPARATOR);
059        String type = keyStr.substring(0, pos);
060        ValueAggregator aggregator = ValueAggregatorBaseDescriptor
061          .generateValueAggregator(type);
062        while (values.hasNext()) {
063          aggregator.addNextValue(values.next());
064        }
065        Iterator outputs = aggregator.getCombinerOutput().iterator();
066    
067        while (outputs.hasNext()) {
068          Object v = outputs.next();
069          if (v instanceof Text) {
070            output.collect(key, (Text)v);
071          } else {
072            output.collect(key, new Text(v.toString()));
073          }
074        }
075      }
076    
077      /** 
078       * Do nothing. 
079       *
080       */
081      public void close() throws IOException {
082    
083      }
084    
085      /** 
086       * Do nothing. Should not be called. 
087       *
088       */
089      public void map(K1 arg0, V1 arg1, OutputCollector<Text, Text> arg2,
090                      Reporter arg3) throws IOException {
091        throw new IOException ("should not be called\n");
092      }
093    }