1 | /** |
---|
2 | * Licensed to the Apache Software Foundation (ASF) under one |
---|
3 | * or more contributor license agreements. See the NOTICE file |
---|
4 | * distributed with this work for additional information |
---|
5 | * regarding copyright ownership. The ASF licenses this file |
---|
6 | * to you under the Apache License, Version 2.0 (the |
---|
7 | * "License"); you may not use this file except in compliance |
---|
8 | * with the License. You may obtain a copy of the License at |
---|
9 | * |
---|
10 | * http://www.apache.org/licenses/LICENSE-2.0 |
---|
11 | * |
---|
12 | * Unless required by applicable law or agreed to in writing, software |
---|
13 | * distributed under the License is distributed on an "AS IS" BASIS, |
---|
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
---|
15 | * See the License for the specific language governing permissions and |
---|
16 | * limitations under the License. |
---|
17 | */ |
---|
18 | package org.apache.hadoop.fs; |
---|
19 | |
---|
20 | import java.io.IOException; |
---|
21 | import java.net.InetAddress; |
---|
22 | import org.apache.hadoop.conf.Configuration; |
---|
23 | import org.apache.hadoop.conf.Configured; |
---|
24 | import org.apache.hadoop.io.LongWritable; |
---|
25 | import org.apache.hadoop.io.UTF8; |
---|
26 | import org.apache.hadoop.io.Writable; |
---|
27 | import org.apache.hadoop.io.WritableComparable; |
---|
28 | import org.apache.hadoop.mapred.JobConf; |
---|
29 | import org.apache.hadoop.mapred.Mapper; |
---|
30 | import org.apache.hadoop.mapred.OutputCollector; |
---|
31 | import org.apache.hadoop.mapred.Reporter; |
---|
32 | |
---|
33 | /** |
---|
34 | * Base mapper class for IO operations. |
---|
35 | * <p> |
---|
36 | * Two abstract method {@link #doIO(Reporter, String, long)} and |
---|
37 | * {@link #collectStats(OutputCollector,String,long,Object)} should be |
---|
38 | * overloaded in derived classes to define the IO operation and the |
---|
39 | * statistics data to be collected by subsequent reducers. |
---|
40 | * |
---|
41 | */ |
---|
42 | public abstract class IOMapperBase extends Configured |
---|
43 | implements Mapper<UTF8, LongWritable, UTF8, UTF8> { |
---|
44 | |
---|
45 | protected byte[] buffer; |
---|
46 | protected int bufferSize; |
---|
47 | protected FileSystem fs; |
---|
48 | protected String hostName; |
---|
49 | |
---|
50 | public IOMapperBase(Configuration conf) { |
---|
51 | super(conf); |
---|
52 | try { |
---|
53 | fs = FileSystem.get(conf); |
---|
54 | } catch (Exception e) { |
---|
55 | throw new RuntimeException("Cannot create file system.", e); |
---|
56 | } |
---|
57 | bufferSize = conf.getInt("test.io.file.buffer.size", 4096); |
---|
58 | buffer = new byte[bufferSize]; |
---|
59 | try { |
---|
60 | hostName = InetAddress.getLocalHost().getHostName(); |
---|
61 | } catch(Exception e) { |
---|
62 | hostName = "localhost"; |
---|
63 | } |
---|
64 | } |
---|
65 | |
---|
66 | public void configure(JobConf job) { |
---|
67 | setConf(job); |
---|
68 | } |
---|
69 | |
---|
70 | public void close() throws IOException { |
---|
71 | } |
---|
72 | |
---|
73 | /** |
---|
74 | * Perform io operation, usually read or write. |
---|
75 | * |
---|
76 | * @param reporter |
---|
77 | * @param name file name |
---|
78 | * @param value offset within the file |
---|
79 | * @return object that is passed as a parameter to |
---|
80 | * {@link #collectStats(OutputCollector,String,long,Object)} |
---|
81 | * @throws IOException |
---|
82 | */ |
---|
83 | abstract Object doIO(Reporter reporter, |
---|
84 | String name, |
---|
85 | long value) throws IOException; |
---|
86 | |
---|
87 | /** |
---|
88 | * Collect stat data to be combined by a subsequent reducer. |
---|
89 | * |
---|
90 | * @param output |
---|
91 | * @param name file name |
---|
92 | * @param execTime IO execution time |
---|
93 | * @param doIOReturnValue value returned by {@link #doIO(Reporter,String,long)} |
---|
94 | * @throws IOException |
---|
95 | */ |
---|
96 | abstract void collectStats(OutputCollector<UTF8, UTF8> output, |
---|
97 | String name, |
---|
98 | long execTime, |
---|
99 | Object doIOReturnValue) throws IOException; |
---|
100 | |
---|
101 | /** |
---|
102 | * Map file name and offset into statistical data. |
---|
103 | * <p> |
---|
104 | * The map task is to get the |
---|
105 | * <tt>key</tt>, which contains the file name, and the |
---|
106 | * <tt>value</tt>, which is the offset within the file. |
---|
107 | * |
---|
108 | * The parameters are passed to the abstract method |
---|
109 | * {@link #doIO(Reporter,String,long)}, which performs the io operation, |
---|
110 | * usually read or write data, and then |
---|
111 | * {@link #collectStats(OutputCollector,String,long,Object)} |
---|
112 | * is called to prepare stat data for a subsequent reducer. |
---|
113 | */ |
---|
114 | public void map(UTF8 key, |
---|
115 | LongWritable value, |
---|
116 | OutputCollector<UTF8, UTF8> output, |
---|
117 | Reporter reporter) throws IOException { |
---|
118 | String name = key.toString(); |
---|
119 | long longValue = value.get(); |
---|
120 | |
---|
121 | reporter.setStatus("starting " + name + " ::host = " + hostName); |
---|
122 | |
---|
123 | long tStart = System.currentTimeMillis(); |
---|
124 | Object statValue = doIO(reporter, name, longValue); |
---|
125 | long tEnd = System.currentTimeMillis(); |
---|
126 | long execTime = tEnd - tStart; |
---|
127 | collectStats(output, name, execTime, statValue); |
---|
128 | |
---|
129 | reporter.setStatus("finished " + name + " ::host = " + hostName); |
---|
130 | } |
---|
131 | } |
---|