├── LICENSE.txt
├── README
├── build.xml
├── netcdf_patch
├── INSTALL
├── netcdf42.diff
└── netcdf43.diff
├── runTest.script
├── src
└── edu
│ └── ucsc
│ └── srl
│ └── damasc
│ └── netcdf
│ ├── NCTool.java
│ ├── NetCDFUtils.java
│ ├── Utils.java
│ ├── combine
│ ├── AverageCombiner.java
│ ├── IdentityCombiner.java
│ ├── MaxCombiner.java
│ ├── MedianCombiner.java
│ ├── SimpleMaxCombiner.java
│ └── SimpleMedianCombiner.java
│ ├── io
│ ├── ArraySpec.java
│ ├── AverageResult.java
│ ├── GroupID.java
│ ├── GroupIDGen.java
│ ├── HolisticResult.java
│ ├── NcHdfsRaf.java
│ ├── Result.java
│ ├── SHFileStatus.java
│ └── input
│ │ ├── ArrayBasedFileInputFormat.java
│ │ ├── ArrayBasedFileSplit.java
│ │ ├── NetCDFFileInputFormat.java
│ │ ├── NetCDFFileSplit.java
│ │ └── NetCDFRecordReader.java
│ ├── map
│ ├── AverageMapper.java
│ ├── IdentityMapper.java
│ ├── MaxMapper.java
│ ├── MedianMapper.java
│ ├── NullMapper.java
│ ├── SimpleMaxMapper.java
│ └── SimpleMedianMapper.java
│ ├── reduce
│ ├── AverageReducer.java
│ ├── IdentityReducer.java
│ ├── MaxReducer.java
│ ├── MedianReducer.java
│ ├── NullReducer.java
│ ├── SimpleMaxReducer.java
│ └── SimpleMedianReducer.java
│ └── tools
│ ├── Average.java
│ ├── Identity.java
│ ├── Max.java
│ └── Median.java
└── tools
├── README
├── build.xml
└── src
└── edu
└── ucsc
└── srl
└── tools
└── NetcdfFileGenerator.java
/LICENSE.txt:
--------------------------------------------------------------------------------
1 | All code in this project, unless otherwise stated, is GPLv2 licensed.
2 | If you are interested in another license, please contact me at buck@soe.ucsc.edu.
3 |
--------------------------------------------------------------------------------
/README:
--------------------------------------------------------------------------------
1 | This represents the code base used to run the experiments for the SC '11 SciHadoop paper.
2 | See our website for links to the paper and more details: http://systems.soe.ucsc.edu/projects/damasc
3 |
4 | The NetCDF patches need to be applied to a 4.2 version of the NetCDF Java library for the SciHadoop code to work. Directions can be found in the netcdf_patches directory.
5 |
6 | Installation instructions can be found on our github page, here https://github.com/four2five/SciHadoop/wiki/How-to-Build-SciHadoop
7 | That link has instructions for installing Hadoop, patching and building NetCDF and running a SciHadoop job
8 |
9 | Questions can be sent to buck@soe.ucsc.edu
10 |
--------------------------------------------------------------------------------
/build.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 |
18 |
19 |
20 |
21 |
22 |
23 |
24 |
25 |
26 |
27 |
28 |
29 |
30 |
31 |
32 |
37 |
38 |
39 |
40 |
41 |
42 |
43 |
44 |
45 |
46 |
47 |
48 |
49 |
50 |
51 |
52 |
53 |
54 |
55 |
56 |
57 |
58 |
59 |
60 |
61 |
62 |
63 |
64 |
65 |
66 |
67 |
68 |
69 |
70 |
71 |
72 |
73 |
74 |
75 |
76 |
77 |
78 |
79 |
80 |
81 |
82 |
83 |
84 |
85 |
86 |
87 |
88 |
89 |
--------------------------------------------------------------------------------
/netcdf_patch/INSTALL:
--------------------------------------------------------------------------------
1 | March 6, 2013
2 | Joe Buck ( buck@soe.ucsc.edu )
3 |
4 | We've also added a patch for NetCDF Java 4.3.15 as netcdf43.diff.
5 | from the "cdm" directory, apply the patch like so:
6 | patch -p2 < netcdf43.diff
7 |
8 | Nov 3, 2011
9 | Joe Buck ( buck@soe.ucsc.edu)
10 |
11 | 1) download a source verison of NetCDF 4.2. This patch was built against the package from Oct 24, 2011
12 |
13 | 2) unzip the NetCDF source somewhere
14 |
15 | 3) copy the netcdf42.diff file into the same directory as "cdm" (normally the "thredds directory").
16 |
17 | 4) patch with this command 'patch -p1 < netcdf42.diff
18 |
19 | 5) this should apply cleanly. If it does, then cd into the cdm directory and build the jar with this command:
20 | 'ant makeMainComplete'
21 |
22 | 6) there should now be a directory under 'cdm' called 'target'. The jar file in there called 'netcdfAll-4.2.jar is likely the one you want. You can try using netcdf-4.2.jar but we just use the "All" variant ant it works out pretty well.
23 |
24 | 7) make sure this jar is in both your CLASSPATH and HADOOP_CLASSPATH (configured in the conf/hadoop-env.sh script in your hadoop installation) for every node.
25 |
--------------------------------------------------------------------------------
/netcdf_patch/netcdf42.diff:
--------------------------------------------------------------------------------
1 | diff -b -r -c cdm//src/main/java/ucar/nc2/iosp/AbstractIOServiceProvider.java ../netcdf2/cdm//src/main/java/ucar/nc2/iosp/AbstractIOServiceProvider.java
2 | *** cdm//src/main/java/ucar/nc2/iosp/AbstractIOServiceProvider.java 2011-07-27 11:12:24.000000000 -0700
3 | --- ../netcdf2/cdm//src/main/java/ucar/nc2/iosp/AbstractIOServiceProvider.java 2011-11-04 17:38:52.494397089 -0700
4 | ***************
5 | *** 100,105 ****
6 | --- 100,115 ----
7 | }
8 |
9 | @Override
10 | + public boolean supportsLocalityInformation() throws IOException {
11 | + throw new IOException("This IOSP does not support getLocalityInformation()");
12 | + }
13 | +
14 | + @Override
15 | + public ArrayLong getLocalityInformation(ucar.nc2.Variable v2, Section section) throws IOException, InvalidRangeException {
16 | + throw new IOException("This IOSP does not support getLocalityInformation()");
17 | + }
18 | +
19 | + @Override
20 | public StructureDataIterator getStructureIterator(Structure s, int bufferSize) throws java.io.IOException {
21 | return null;
22 | }
23 | diff -b -r -c cdm//src/main/java/ucar/nc2/iosp/IOServiceProvider.java ../netcdf2/cdm//src/main/java/ucar/nc2/iosp/IOServiceProvider.java
24 | *** cdm//src/main/java/ucar/nc2/iosp/IOServiceProvider.java 2011-07-27 11:12:32.000000000 -0700
25 | --- ../netcdf2/cdm//src/main/java/ucar/nc2/iosp/IOServiceProvider.java 2011-11-04 17:34:34.254397234 -0700
26 | ***************
27 | *** 124,129 ****
28 | --- 124,155 ----
29 | */
30 | public ucar.ma2.Array readSection(ParsedSectionSpec cer) throws IOException, InvalidRangeException;
31 |
32 | + /**
33 | + * Calculate the offset in the underlying byte-stream for each value
34 | + * indicated by the section and variable objects
35 | + *
36 | + * @param v2 a top-level Variable
37 | + * @param section the section of data to read.
38 | + * There must be a Range for each Dimension in the variable, in order.
39 | + * Note: no nulls allowed. IOSP may not modify.
40 | + * @return an ArrayLong the same shape as section where each entry is the
41 | + * offset of the corresponding data element
42 | + * @throws java.io.IOException if read error or if this IOSP doesn't
43 | + * support getLocalityInformation()
44 | + * @throws ucar.ma2.InvalidRangeException if invalid section
45 | + */
46 | + public ucar.ma2.ArrayLong getLocalityInformation(ucar.nc2.Variable v2, Section section)
47 | + throws IOException, InvalidRangeException;
48 | +
49 | + /**
50 | + * Determine if this IOServiceProvider supports the getLocalityInformation() function
51 | + *
52 | + * @return a boolean indicating if getLocalityInformation is supported by
53 | + * this IOServiceProvider
54 | + */
55 | + public boolean supportsLocalityInformation() throws IOException;
56 | +
57 | +
58 | // iosps with top level sequences must override
59 | public StructureDataIterator getStructureIterator(Structure s, int bufferSize) throws java.io.IOException;
60 |
61 | diff -b -r -c cdm//src/main/java/ucar/nc2/iosp/netcdf3/N3iosp.java ../netcdf2/cdm//src/main/java/ucar/nc2/iosp/netcdf3/N3iosp.java
62 | *** cdm//src/main/java/ucar/nc2/iosp/netcdf3/N3iosp.java 2011-07-27 11:12:30.000000000 -0700
63 | --- ../netcdf2/cdm//src/main/java/ucar/nc2/iosp/netcdf3/N3iosp.java 2011-11-04 17:34:34.284397234 -0700
64 | ***************
65 | *** 410,415 ****
66 | --- 410,483 ----
67 | this.fill = fill;
68 | }
69 |
70 | + /**
71 | + * query to see if this IOSP supports the getLocalityInformation() function
72 | + *
73 | + * @return a boolean indicating that this IOSP supports the getLocalityInformation() call
74 | + */
75 | + public boolean supportsLocalityInformation() {
76 | + return true;
77 | + }
78 | +
79 | + /**
80 | + * Returns an ArrayLong with each entry corresponding to the offset in the filestream
81 | + * of the same data cell in the section arguement to the function
82 | + *
83 | + * @param v2 the variable to get the data from
84 | + * @param section the record range to read
85 | + * @return an ArrayLong object that's the shape as the section arguement
86 | + * @throws InvalidRangeException on error
87 | + * @throws IOException on error
88 | + */
89 | + public ArrayLong getLocalityInformation(ucar.nc2.Variable v2, Section section)
90 | + throws InvalidRangeException, IOException {
91 | +
92 | + // An array to hold the offsets that will be returned
93 | + ArrayLong array = new ArrayLong(section.getShape());
94 | +
95 | + // Index into the results array
96 | + Index aIndex = array.getIndex();
97 | +
98 | + // dataSize is used to increment the offsets within a given
99 | + // chunk appropriately
100 | + DataType type = v2.getDataType();
101 | + int dataSize = type.getSize();
102 | +
103 | + Layout layout = getLayout(v2, section);
104 | +
105 | + // iterate over all the chunks in the calculated Layout
106 | + while( layout.hasNext() ){
107 | + Layout.Chunk chunk = layout.next();
108 | +
109 | + // iterate over the elements in this chunk
110 | + for( int i = 0; i < chunk.getNelems(); i++){
111 | + // write the offset into the results array, then iterate the index
112 | + array.setLong(aIndex, chunk.getSrcPos() + (i * dataSize));
113 | + aIndex.incr();
114 | + }
115 | + }
116 | +
117 | + return array;
118 | + }
119 | +
120 | + /**
121 | + * Returns a Layout object for use by an N3iosp object
122 | + *
123 | + * @param v2 the variable to get the layout information for
124 | + * @param section the record range to read
125 | + * @return a Layout corresponding to the Section requested
126 | + * @throws IOException on error
127 | + */
128 | + private Layout getLayout(Variable v2, Section section) throws InvalidRangeException {
129 | + N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
130 | +
131 | + Layout layout = (!v2.isUnlimited()) ? new LayoutRegular(vinfo.begin, v2.getElementSize(), v2.getShape(), section)
132 | + : new LayoutRegularSegmented(vinfo.begin, v2.getElementSize(), header.recsize, v2.getShape(), section);
133 | +
134 | + return layout;
135 | + }
136 | +
137 | + /////////////////////////////////////////////////////////////////////////////
138 | /////////////////////////////////////////////////////////////////////////////
139 | // data reading
140 |
141 | ***************
142 | *** 419,427 ****
143 |
144 | N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
145 | DataType dataType = v2.getDataType();
146 | !
147 | ! Layout layout = (!v2.isUnlimited()) ? new LayoutRegular(vinfo.begin, v2.getElementSize(), v2.getShape(), section) :
148 | ! new LayoutRegularSegmented(vinfo.begin, v2.getElementSize(), header.recsize, v2.getShape(), section);
149 |
150 | if (layout.getTotalNelems() == 0) {
151 | return Array.factory(dataType.getPrimitiveClassType(), section.getShape());
152 | --- 487,493 ----
153 |
154 | N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
155 | DataType dataType = v2.getDataType();
156 | ! Layout layout = getLayout(v2, section);
157 |
158 | if (layout.getTotalNelems() == 0) {
159 | return Array.factory(dataType.getPrimitiveClassType(), section.getShape());
160 | ***************
161 | *** 565,573 ****
162 |
163 | N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
164 | DataType dataType = v2.getDataType();
165 | !
166 | ! Layout layout = (!v2.isUnlimited()) ? new LayoutRegular(vinfo.begin, v2.getElementSize(), v2.getShape(), section) :
167 | ! new LayoutRegularSegmented(vinfo.begin, v2.getElementSize(), header.recsize, v2.getShape(), section);
168 |
169 | return readData(layout, dataType, channel);
170 | }
171 | --- 631,637 ----
172 |
173 | N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
174 | DataType dataType = v2.getDataType();
175 | ! Layout layout = getLayout(v2, section);
176 |
177 | return readData(layout, dataType, channel);
178 | }
179 | ***************
180 | *** 663,669 ****
181 |
182 | public void writeData(Variable v2, Section section, Array values) throws java.io.IOException, InvalidRangeException {
183 |
184 | - N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
185 | DataType dataType = v2.getDataType();
186 |
187 | if (v2.isUnlimited()) {
188 | --- 727,732 ----
189 | ***************
190 | *** 675,682 ****
191 | writeRecordData((Structure) v2, section, values);
192 |
193 | } else {
194 | ! Layout layout = (!v2.isUnlimited()) ? new LayoutRegular(vinfo.begin, v2.getElementSize(), v2.getShape(), section) :
195 | ! new LayoutRegularSegmented(vinfo.begin, v2.getElementSize(), header.recsize, v2.getShape(), section);
196 | writeData(values, layout, dataType);
197 | }
198 | }
199 | --- 738,744 ----
200 | writeRecordData((Structure) v2, section, values);
201 |
202 | } else {
203 | ! Layout layout = getLayout(v2, section);
204 | writeData(values, layout, dataType);
205 | }
206 | }
207 | diff -b -r -c cdm//src/main/java/ucar/nc2/NetcdfFile.java ../netcdf2/cdm//src/main/java/ucar/nc2/NetcdfFile.java
208 | *** cdm//src/main/java/ucar/nc2/NetcdfFile.java 2011-07-27 11:12:34.000000000 -0700
209 | --- ../netcdf2/cdm//src/main/java/ucar/nc2/NetcdfFile.java 2011-11-04 17:34:34.224397234 -0700
210 | ***************
211 | *** 423,428 ****
212 | --- 423,432 ----
213 | return open(location, null);
214 | }
215 |
216 | + public static NetcdfFile open(ucar.unidata.io.RandomAccessFile raf, String location) throws IOException {
217 | + return open(raf, location, null, null);
218 | + }
219 | +
220 | /**
221 | * Open an existing file (read only), with option of cancelling.
222 | *
223 | ***************
224 | *** 1094,1099 ****
225 | --- 1098,1116 ----
226 | }
227 |
228 | /**
229 | + * Return true if this file supports querying for locality information.
230 | + * @return if the file can return locality information
231 | + * @throws IOException if there is not an IOServiceProvider set
232 | + */
233 | + public boolean supportsLocalityInformation() throws IOException {
234 | + if (spi == null) {
235 | + throw new IOException("spi is null");
236 | + }
237 | +
238 | + return spi.supportsLocalityInformation();
239 | + }
240 | +
241 | + /**
242 | * Return the unlimited (record) dimension, or null if not exist.
243 | * If there are multiple unlimited dimensions, it will return the first one.
244 | *
245 | ***************
246 | *** 1834,1839 ****
247 | --- 1851,1867 ----
248 | return sbuff.toString();
249 | }
250 |
251 | + public ArrayLong getLocalityInformation(ucar.nc2.Variable v, Section ranges)
252 | + throws IOException, InvalidRangeException {
253 | + if (spi == null) {
254 | + throw new IOException("spi is null");
255 | + } else if (!spi.supportsLocalityInformation()) {
256 | + throw new IOException("IOSP " + spi.getFileTypeId() + " does not support getLocalityInformation()");
257 | + }
258 | +
259 | + return spi.getLocalityInformation(v, ranges);
260 | + }
261 | +
262 | //////////////////////////////////////////////////////////////////////////////////////
263 | // Service Provider calls
264 | // All IO eventually goes through these calls.
265 | diff -b -r -c cdm//src/main/java/ucar/nc2/Variable.java ../netcdf2/cdm//src/main/java/ucar/nc2/Variable.java
266 | *** cdm//src/main/java/ucar/nc2/Variable.java 2011-07-27 11:12:34.000000000 -0700
267 | --- ../netcdf2/cdm//src/main/java/ucar/nc2/Variable.java 2011-11-04 17:34:34.104397234 -0700
268 | ***************
269 | *** 605,610 ****
270 | --- 605,644 ----
271 | }
272 |
273 | /**
274 | + * Return an ArrayLong structure with the same shape as the shape arguement
275 | + * where each entry is the offset in the byte-stream where the corresponding
276 | + * data element begins
277 | + * @param origin int array specifying the starting index. If null, assume all zeroes.
278 | + * @param shape int array specifying the extents in each dimension.
279 | + * This becomes the shape of the returned Array.
280 | + * @return a memory-resdient array of longs with offsets or -1 if the offset
281 | + * for the corresponding cell can't be determined
282 | + */
283 | + public ArrayLong getLocalityInformation(int[] origin, int[] shape) throws IOException, InvalidRangeException {
284 | + return ncfile.getLocalityInformation( this, new Section(origin, shape));
285 | + }
286 | +
287 | + public boolean supportsLocalityInformation() throws IOException {
288 | + return ncfile.supportsLocalityInformation();
289 | + }
290 | +
291 | + /**
292 | + * Return an ArrayLong structure with the same shape as the shape arguement
293 | + * where each entry is the offset in the byte-stream where the corresponding
294 | + * data element begins.
295 | + *
296 | + * @param section list of Range specifying the section of data to read.
297 | + * Must be null or same rank as variable.
298 | + * If list is null, assume all data.
299 | + * Each Range corresponds to a Dimension. If the Range object is null, it means use the entire dimension.
300 | + * @return a memory-resdient array of longs with offsets or -1 if the offset
301 | + * for the corresponding cell can't be determined
302 | + */
303 | + public ArrayLong getLocalityInformation(Section section) throws IOException, InvalidRangeException {
304 | + return ncfile.getLocalityInformation(this, section);
305 | + }
306 | +
307 | + /**
308 | * Read data section specified by a "section selector", and return a memory resident Array. Uses
309 | * Fortran 90 array section syntax.
310 | *
311 | Only in ../netcdf2/cdm/: target
312 |
--------------------------------------------------------------------------------
/netcdf_patch/netcdf43.diff:
--------------------------------------------------------------------------------
1 | diff -Naur thredds2/cdm/src/main/java/ucar/nc2/iosp/AbstractIOServiceProvider.java thredds/cdm/src/main/java/ucar/nc2/iosp/AbstractIOServiceProvider.java
2 | --- thredds2/cdm/src/main/java/ucar/nc2/iosp/AbstractIOServiceProvider.java 2013-03-05 16:21:20.341011771 -0800
3 | +++ thredds/cdm/src/main/java/ucar/nc2/iosp/AbstractIOServiceProvider.java 2013-03-05 15:17:24.793231134 -0800
4 | @@ -152,4 +152,13 @@
5 | return "N/A";
6 | }
7 |
8 | + public boolean supportsLocalityInformation() {
9 | + return false;
10 | + }
11 | +
12 | + public ArrayLong getLocalityInformation(ucar.nc2.Variable v2, Section section)
13 | + throws InvalidRangeException, IOException {
14 | + return null;
15 | + }
16 | +
17 | }
18 | diff -Naur thredds2/cdm/src/main/java/ucar/nc2/iosp/IOServiceProvider.java thredds/cdm/src/main/java/ucar/nc2/iosp/IOServiceProvider.java
19 | --- thredds2/cdm/src/main/java/ucar/nc2/iosp/IOServiceProvider.java 2013-03-05 16:21:20.345011720 -0800
20 | +++ thredds/cdm/src/main/java/ucar/nc2/iosp/IOServiceProvider.java 2013-03-05 14:55:08.786027020 -0800
21 | @@ -131,6 +131,32 @@
22 | */
23 | public ucar.ma2.Array readSection(ParsedSectionSpec cer) throws IOException, InvalidRangeException;
24 |
25 | + /**
26 | + * Calculate the offset in the underlying byte-stream for each value
27 | + * indicated by the section and variable objects
28 | + *
29 | + * @param v2 a top-level Variable
30 | + * @param section the section of data to read.
31 | + * There must be a Range for each Dimension in the variable, in order.
32 | + * Note: no nulls allowed. IOSP may not modify.
33 | + * @return an ArrayLong the same shape as section where each entry is the
34 | + * offset of the corresponding data element
35 | + * @throws java.io.IOException if read error or if this IOSP doesn't
36 | + * support getLocalityInformation()
37 | + * @throws ucar.ma2.InvalidRangeException if invalid section
38 | + */
39 | + public ucar.ma2.ArrayLong getLocalityInformation(ucar.nc2.Variable v2, Section section)
40 | + throws IOException, InvalidRangeException;
41 | +
42 | + /**
43 | + * Determine if this IOServiceProvider supports the getLocalityInformation() function
44 | + *
45 | + * @return a boolean indicating if getLocalityInformation is supported by
46 | + * this IOServiceProvider
47 | + */
48 | + public boolean supportsLocalityInformation() throws IOException;
49 | +
50 | +
51 | // iosps with top level sequences must override
52 | public StructureDataIterator getStructureIterator(Structure s, int bufferSize) throws java.io.IOException;
53 |
54 | diff -Naur thredds2/cdm/src/main/java/ucar/nc2/iosp/netcdf3/N3iosp.java thredds/cdm/src/main/java/ucar/nc2/iosp/netcdf3/N3iosp.java
55 | --- thredds2/cdm/src/main/java/ucar/nc2/iosp/netcdf3/N3iosp.java 2013-03-05 16:21:20.381011267 -0800
56 | +++ thredds/cdm/src/main/java/ucar/nc2/iosp/netcdf3/N3iosp.java 2013-03-05 14:56:54.320700269 -0800
57 | @@ -433,6 +433,74 @@
58 | this.fill = fill;
59 | }
60 |
61 | + /**
62 | + * query to see if this IOSP supports the getLocalityInformation() function
63 | + *
64 | + * @return a boolean indicating that this IOSP supports the getLocalityInformation() call
65 | + */
66 | + public boolean supportsLocalityInformation() {
67 | + return true;
68 | + }
69 | +
70 | + /**
71 | + * Returns an ArrayLong with each entry corresponding to the offset in the filestream
72 | + * of the same data cell in the section arguement to the function
73 | + *
74 | + * @param v2 the variable to get the data from
75 | + * @param section the record range to read
76 | + * @return an ArrayLong object that's the shape as the section arguement
77 | + * @throws InvalidRangeException on error
78 | + * @throws IOException on error
79 | + */
80 | + public ArrayLong getLocalityInformation(ucar.nc2.Variable v2, Section section)
81 | + throws InvalidRangeException, IOException {
82 | +
83 | + // An array to hold the offsets that will be returned
84 | + ArrayLong array = new ArrayLong(section.getShape());
85 | +
86 | + // Index into the results array
87 | + Index aIndex = array.getIndex();
88 | +
89 | + // dataSize is used to increment the offsets within a given
90 | + // chunk appropriately
91 | + DataType type = v2.getDataType();
92 | + int dataSize = type.getSize();
93 | +
94 | + Layout layout = getLayout(v2, section);
95 | +
96 | + // iterate over all the chunks in the calculated Layout
97 | + while( layout.hasNext() ){
98 | + Layout.Chunk chunk = layout.next();
99 | +
100 | + // iterate over the elements in this chunk
101 | + for( int i = 0; i < chunk.getNelems(); i++){
102 | + // write the offset into the results array, then iterate the index
103 | + array.setLong(aIndex, chunk.getSrcPos() + (i * dataSize));
104 | + aIndex.incr();
105 | + }
106 | + }
107 | +
108 | + return array;
109 | + }
110 | +
111 | + /**
112 | + * Returns a Layout object for use by an N3iosp object
113 | + *
114 | + * @param v2 the variable to get the layout information for
115 | + * @param section the record range to read
116 | + * @return a Layout corresponding to the Section requested
117 | + * @throws IOException on error
118 | + */
119 | + private Layout getLayout(Variable v2, Section section) throws InvalidRangeException {
120 | + N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
121 | +
122 | + Layout layout = (!v2.isUnlimited()) ? new LayoutRegular(vinfo.begin, v2.getElementSize(), v2.getShape(), section)
123 | + : new LayoutRegularSegmented(vinfo.begin, v2.getElementSize(), header.recsize, v2.getShape(), section);
124 | +
125 | + return layout;
126 | + }
127 | +
128 | + /////////////////////////////////////////////////////////////////////////////
129 | /////////////////////////////////////////////////////////////////////////////
130 | // data reading
131 |
132 | @@ -442,9 +510,7 @@
133 |
134 | N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
135 | DataType dataType = v2.getDataType();
136 | -
137 | - Layout layout = (!v2.isUnlimited()) ? new LayoutRegular(vinfo.begin, v2.getElementSize(), v2.getShape(), section) :
138 | - new LayoutRegularSegmented(vinfo.begin, v2.getElementSize(), header.recsize, v2.getShape(), section);
139 | + Layout layout = getLayout(v2, section);
140 |
141 | if (layout.getTotalNelems() == 0) {
142 | return Array.factory(dataType.getPrimitiveClassType(), section.getShape());
143 | @@ -588,9 +654,7 @@
144 |
145 | N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
146 | DataType dataType = v2.getDataType();
147 | -
148 | - Layout layout = (!v2.isUnlimited()) ? new LayoutRegular(vinfo.begin, v2.getElementSize(), v2.getShape(), section) :
149 | - new LayoutRegularSegmented(vinfo.begin, v2.getElementSize(), header.recsize, v2.getShape(), section);
150 | + Layout layout = getLayout(v2, section);
151 |
152 | return readData(layout, dataType, channel);
153 | }
154 | @@ -669,7 +733,7 @@
155 | // write
156 |
157 | public void writeData(Variable v2, Section section, Array values) throws java.io.IOException, InvalidRangeException {
158 | - N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
159 | + //N3header.Vinfo vinfo = (N3header.Vinfo) v2.getSPobject();
160 | DataType dataType = v2.getDataType();
161 |
162 | if (v2.isUnlimited()) {
163 | @@ -681,8 +745,7 @@
164 | writeRecordData((Structure) v2, section, values);
165 |
166 | } else {
167 | - Layout layout = (!v2.isUnlimited()) ? new LayoutRegular(vinfo.begin, v2.getElementSize(), v2.getShape(), section) :
168 | - new LayoutRegularSegmented(vinfo.begin, v2.getElementSize(), header.recsize, v2.getShape(), section);
169 | + Layout layout = getLayout(v2, section);
170 | writeData(values, layout, dataType);
171 | }
172 | }
173 | diff -Naur thredds2/cdm/src/main/java/ucar/nc2/NetcdfFile.java thredds/cdm/src/main/java/ucar/nc2/NetcdfFile.java
174 | --- thredds2/cdm/src/main/java/ucar/nc2/NetcdfFile.java 2013-03-05 16:21:20.229013179 -0800
175 | +++ thredds/cdm/src/main/java/ucar/nc2/NetcdfFile.java 2013-03-05 14:55:08.790026970 -0800
176 | @@ -441,6 +441,10 @@
177 | return open(location, null);
178 | }
179 |
180 | + public static NetcdfFile open(ucar.unidata.io.RandomAccessFile raf, String location) throws IOException {
181 | + return open(raf, location, null, null);
182 | + }
183 | +
184 | /**
185 | * Open an existing file (read only), with option of cancelling.
186 | *
187 | @@ -1159,6 +1163,19 @@
188 | }
189 |
190 | /**
191 | + * Return true if this file supports querying for locality information.
192 | + * @return if the file can return locality information
193 | + * @throws IOException if there is not an IOServiceProvider set
194 | + */
195 | + public boolean supportsLocalityInformation() throws IOException {
196 | + if (spi == null) {
197 | + throw new IOException("spi is null");
198 | + }
199 | +
200 | + return spi.supportsLocalityInformation();
201 | + }
202 | +
203 | + /**
204 | * Return the unlimited (record) dimension, or null if not exist.
205 | * If there are multiple unlimited dimensions, it will return the first one.
206 | *
207 | @@ -1910,6 +1927,17 @@
208 | return makeValidCdmObjectName(sbuff.toString());
209 | }
210 |
211 | + public ArrayLong getLocalityInformation(ucar.nc2.Variable v, Section ranges)
212 | + throws IOException, InvalidRangeException {
213 | + if (spi == null) {
214 | + throw new IOException("spi is null");
215 | + } else if (!spi.supportsLocalityInformation()) {
216 | + throw new IOException("IOSP " + spi.getFileTypeId() + " does not support getLocalityInformation()");
217 | + }
218 | +
219 | + return spi.getLocalityInformation(v, ranges);
220 | + }
221 | +
222 | //////////////////////////////////////////////////////////////////////////////////////
223 | // Service Provider calls
224 | // All IO eventually goes through these calls.
225 | diff -Naur thredds2/cdm/src/main/java/ucar/nc2/Variable.java thredds/cdm/src/main/java/ucar/nc2/Variable.java
226 | --- thredds2/cdm/src/main/java/ucar/nc2/Variable.java 2013-03-05 16:21:20.233013128 -0800
227 | +++ thredds/cdm/src/main/java/ucar/nc2/Variable.java 2013-03-05 14:55:08.790026970 -0800
228 | @@ -642,6 +642,40 @@
229 | }
230 |
231 | /**
232 | + * Return an ArrayLong structure with the same shape as the shape arguement
233 | + * where each entry is the offset in the byte-stream where the corresponding
234 | + * data element begins
235 | + * @param origin int array specifying the starting index. If null, assume all zeroes.
236 | + * @param shape int array specifying the extents in each dimension.
237 | + * This becomes the shape of the returned Array.
238 | + * @return a memory-resdient array of longs with offsets or -1 if the offset
239 | + * for the corresponding cell can't be determined
240 | + */
241 | + public ArrayLong getLocalityInformation(int[] origin, int[] shape) throws IOException, InvalidRangeException {
242 | + return ncfile.getLocalityInformation( this, new Section(origin, shape));
243 | + }
244 | +
245 | + public boolean supportsLocalityInformation() throws IOException {
246 | + return ncfile.supportsLocalityInformation();
247 | + }
248 | +
249 | + /**
250 | + * Return an ArrayLong structure with the same shape as the shape arguement
251 | + * where each entry is the offset in the byte-stream where the corresponding
252 | + * data element begins.
253 | + *
254 | + * @param section list of Range specifying the section of data to read.
255 | + * Must be null or same rank as variable.
256 | + * If list is null, assume all data.
257 | + * Each Range corresponds to a Dimension. If the Range object is null, it means use the entire dimension.
258 | + * @return a memory-resdient array of longs with offsets or -1 if the offset
259 | + * for the corresponding cell can't be determined
260 | + */
261 | + public ArrayLong getLocalityInformation(Section section) throws IOException, InvalidRangeException {
262 | + return ncfile.getLocalityInformation(this, section);
263 | + }
264 | +
265 | + /**
266 | * Read data section specified by a "section selector", and return a memory resident Array. Uses
267 | * Fortran 90 array section syntax.
268 | *
269 |
--------------------------------------------------------------------------------
/runTest.script:
--------------------------------------------------------------------------------
1 | hadoop jar build/jar/hadoop-scidata.jar average -D damasc.extraction_shape=1,36,36,10 -D damasc.partition_mode=record -D damasc.placement_mode=sampling -D damasc.query_dependant=false -D damasc.number_reducers=1 -D damasc.variable_name=windspeed1 -D damasc.buffer_size=134217728 -D damasc.logfile=/tmp/damasc_log.txt test_data/* test_output
2 |
3 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/NCTool.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf;
2 |
3 | import org.apache.hadoop.util.ProgramDriver;
4 |
5 | import edu.ucsc.srl.damasc.netcdf.tools.Identity;
6 | import edu.ucsc.srl.damasc.netcdf.tools.Average;
7 | import edu.ucsc.srl.damasc.netcdf.tools.Max;
8 | import edu.ucsc.srl.damasc.netcdf.tools.Median;
9 |
10 | /**
11 | * Helper class that registers the various functions that
12 | * SciHadoop supports
13 | */
14 | public class NCTool {
15 | public static void main(String[] args) {
16 | int exitCode = -1;
17 | ProgramDriver pgd = new ProgramDriver();
18 | try {
19 | pgd.addClass("average", Average.class, "NetCDF average job");
20 | pgd.addClass("identity", Identity.class, "NetCDF identity job");
21 | pgd.addClass("max", Max.class, "NetCDF max job");
22 | pgd.addClass("median", Median.class, "NetCDF median job");
23 |
24 | //exitCode = pgd.driver(args);
25 | pgd.driver(args);
26 | } catch (Throwable e) {
27 | e.printStackTrace();
28 | }
29 |
30 | System.exit(exitCode);
31 | }
32 | }
33 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/NetCDFUtils.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf;
2 |
3 | import java.io.BufferedWriter;
4 | import java.io.File;
5 | import java.io.FileWriter;
6 | import java.io.IOException;
7 | import java.util.HashMap;
8 | import java.util.Map;
9 | import java.util.Set;
10 |
11 | import org.apache.commons.logging.Log;
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
14 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
15 |
16 | import ucar.ma2.Array;
17 |
18 | /**
19 | * Utility methods that are specific to NetCDF files / data
20 | */
21 | public class NetCDFUtils {
22 |
23 | /**
24 | * Logs the IDs generated for a given Key / Value pair
25 | * (ArraySpec / Array objects)
26 | * @param debugFileName file to log this data to
27 | * @param ncArray a NetCDF Array object that is the data
28 | * for the "value" part of the
29 | * key / value pair
30 | * @param key an ArraySpec object that is the key
31 | * for the "key" value pair. An ArraySpec
32 | * @param extractionShape the extraction shape specified
33 | * for this query
34 | * @param groupSubArrayMap mapping from GroupIDs to data
35 | * @param LOG the log object to write to in case of an exception
36 | */
37 | public static void logGIDs( String debugFileName, Array ncArray,
38 | ArraySpec key, int[] extractionShape,
39 | HashMap groupSubArrayMap,
40 | Log LOG) {
41 | try {
42 | File outputFile = new File( debugFileName );
43 | BufferedWriter writer =
44 | new BufferedWriter( new FileWriter(outputFile, true));
45 |
46 | Set> set = groupSubArrayMap.entrySet();
47 |
48 | writer.write("InputSplit: " + key);
49 | writer.newLine();
50 | for( Map.Entry me : set ) {
51 | writer.write("\tgid: " + me.getKey().toString(extractionShape) +
52 | "\tspec: " + Utils.arrayToString(me.getValue().getShape()) );
53 | writer.newLine();
54 | }
55 | writer.close();
56 |
57 | } catch ( IOException ioe ) {
58 | LOG.error("Caught an ioe in MedianMapper.logGIDS()\n" + ioe.toString());
59 | }
60 | }
61 | }
62 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/combine/AverageCombiner.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.combine;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.Text;
9 | import org.apache.hadoop.io.IntWritable;
10 | import org.apache.hadoop.io.LongWritable;
11 | import org.apache.hadoop.mapreduce.Reducer;
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
14 | import edu.ucsc.srl.damasc.netcdf.io.AverageResult;
15 |
16 | /**
17 | * Combiner class for the Average function
18 | */
19 | public class AverageCombiner extends
20 | Reducer {
21 |
22 | private static final Log LOG = LogFactory.getLog(AverageCombiner.class);
23 |
24 | /**
25 | * Reduces values for a given key
26 | * @param key the Key for the given values being passed in
27 | * @param values a List of AverageResult objects to combine
28 | * @param context the Context object for the currently executing job
29 | */
30 | public void reduce(LongWritable key, Iterable values,
31 | Context context)
32 | throws IOException, InterruptedException {
33 |
34 |
35 | AverageResult avgResult = new AverageResult();
36 |
37 | for (AverageResult value : values) {
38 | avgResult.addAverageResult(value);
39 | }
40 |
41 | context.write(key, avgResult);
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/combine/IdentityCombiner.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.combine;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.IntWritable;
9 | import org.apache.hadoop.io.LongWritable;
10 | import org.apache.hadoop.mapreduce.Reducer;
11 |
12 | /**
13 | * A combiner that finds the max value for a given key
14 | */
15 | public class IdentityCombiner extends
16 | Reducer {
17 |
18 | private static final Log LOG = LogFactory.getLog(IdentityCombiner.class);
19 |
20 | /**
21 | * Reduces values for a given key
22 | * @param key the Key for the given values being passed in
23 | * @param values a List of IntWritable objects to combine
24 | * @param context the Context object for the currently executing job
25 | */
26 | public void reduce(LongWritable key, Iterable values,
27 | Context context)
28 | throws IOException, InterruptedException {
29 | long timer = System.currentTimeMillis();
30 |
31 |
32 | IntWritable maxVal = new IntWritable();
33 | maxVal.set(Integer.MIN_VALUE);
34 |
35 | for (IntWritable value : values) {
36 | if ( value.get() > maxVal.get() )
37 | maxVal.set(value.get());
38 |
39 | }
40 |
41 | context.write(key, maxVal);
42 | timer = System.currentTimeMillis() - timer;
43 |
44 | LOG.info("Entire combiner took " + timer + " ms");
45 |
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/combine/MaxCombiner.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.combine;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.Text;
9 | import org.apache.hadoop.io.IntWritable;
10 | import org.apache.hadoop.io.LongWritable;
11 | import org.apache.hadoop.mapreduce.Reducer;
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
14 | import edu.ucsc.srl.damasc.netcdf.io.Result;
15 |
16 | /**
17 | * Combiner for the max operator
18 | */
19 | public class MaxCombiner extends
20 | Reducer {
21 |
22 | private static final Log LOG = LogFactory.getLog(MaxCombiner.class);
23 |
24 | /**
25 | * Reduces values for a given key
26 | * @param key the Key for the given values being passed in
27 | * @param values a List of IntWritable objects to combine
28 | * @param context the Context object for the currently executing job
29 | */
30 | public void reduce(GroupID key, Iterable values,
31 | Context context)
32 | throws IOException, InterruptedException {
33 |
34 |
35 | IntWritable maxVal = new IntWritable();
36 | maxVal.set(Integer.MIN_VALUE);
37 |
38 | for (IntWritable value : values) {
39 | if ( value.get() > maxVal.get() )
40 | maxVal.set(value.get());
41 | }
42 |
43 | context.write(key, maxVal);
44 | }
45 | }
46 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/combine/MedianCombiner.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.combine;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.Text;
9 | import org.apache.hadoop.io.IntWritable;
10 | import org.apache.hadoop.io.LongWritable;
11 | import org.apache.hadoop.mapreduce.Reducer;
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
14 | import edu.ucsc.srl.damasc.netcdf.io.HolisticResult;
15 | import edu.ucsc.srl.damasc.netcdf.io.Result;
16 |
17 | import edu.ucsc.srl.damasc.netcdf.Utils;
18 |
19 | /**
20 | * Combiner class for the Median operator
21 | */
22 | public class MedianCombiner extends
23 | Reducer {
24 |
25 | private static final Log LOG = LogFactory.getLog(MedianCombiner.class);
26 | static enum MedianCombinerStatus { FULL, NOTFULL, MERGED }
27 |
28 | /**
29 | * Reduces values for a given key
30 | * @param key the Key for the given values being passed in
31 | * @param values a List of HolisticResult objects to combine
32 | * @param context the Context object for the currently executing job
33 | */
34 |
35 | public void reduce(GroupID key, Iterable values,
36 | Context context)
37 | throws IOException, InterruptedException {
38 |
39 |
40 | // now we need to parse the variable dimensions out
41 | int[] variableShape = Utils.getVariableShape( context.getConfiguration());
42 | int[] extractionShape =
43 | Utils.getExtractionShape(context.getConfiguration(),
44 | variableShape.length);
45 | int neededValues = Utils.calcTotalSize(extractionShape);
46 | GroupID tempID = new GroupID();
47 |
48 | HolisticResult holVal = new HolisticResult();
49 | holVal.setNeededValueCount( neededValues );
50 |
51 | for (HolisticResult value : values) {
52 | if ( holVal.isFull() ) {
53 | LOG.warn("Adding an element to an already full HR. Key: " +
54 | key.toString() +
55 | " array size: " + holVal.getNeededValueCount() +
56 | " current elems: " +
57 | holVal.getCurrentValueCount() );
58 | }
59 |
60 | holVal.merge(value);
61 | context.getCounter(MedianCombinerStatus.MERGED).increment(value.getCurrentValueCount());
62 | }
63 |
64 | // now, the remainig holistic result should be full. Check though
65 | if( holVal.isFull() ) {
66 | // apply whatever function you want, in this case we
67 | // sort and then pull the median out
68 | holVal.sort();
69 | holVal.setFinal( holVal.getValues()[(holVal.getValues().length)/2] );
70 | context.getCounter(MedianCombinerStatus.FULL).increment(1);
71 | } else {
72 | context.getCounter(MedianCombinerStatus.NOTFULL).increment(1);
73 | }
74 |
75 | context.write(key, holVal);
76 | }
77 | }
78 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/combine/SimpleMaxCombiner.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.combine;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.Text;
9 | import org.apache.hadoop.io.IntWritable;
10 | import org.apache.hadoop.io.LongWritable;
11 | import org.apache.hadoop.mapreduce.Reducer;
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
14 | import edu.ucsc.srl.damasc.netcdf.io.Result;
15 |
16 | /**
17 | * Combiner for the Max function that uses simple data structures as keys
18 | * and values
19 | */
20 | public class SimpleMaxCombiner extends
21 | Reducer {
22 |
23 | private static final Log LOG = LogFactory.getLog(SimpleMaxCombiner.class);
24 |
25 | /**
26 | * Reduces values for a given key
27 | * @param key the Key for the given values being passed in
28 | * @param values a List of IntWritable objects to combine
29 | * @param context the Context object for the currently executing job
30 | */
31 | public void reduce(LongWritable key, Iterable values,
32 | Context context)
33 | throws IOException, InterruptedException {
34 |
35 |
36 | IntWritable maxVal = new IntWritable();
37 | maxVal.set(Integer.MIN_VALUE);
38 |
39 | //for (Result value : values) {
40 | for (IntWritable value : values) {
41 | if ( value.get() > maxVal.get() )
42 | maxVal.set(value.get());
43 | }
44 | context.write(key, maxVal);
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/combine/SimpleMedianCombiner.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.combine;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.Text;
9 | import org.apache.hadoop.io.IntWritable;
10 | import org.apache.hadoop.io.LongWritable;
11 | import org.apache.hadoop.mapreduce.Reducer;
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
14 | import edu.ucsc.srl.damasc.netcdf.io.HolisticResult;
15 | import edu.ucsc.srl.damasc.netcdf.io.Result;
16 |
17 | import edu.ucsc.srl.damasc.netcdf.Utils;
18 |
19 | /**
20 | * Combiner class for the Median operator that uses simple data structures for
21 | * keys and values
22 | */
23 | public class SimpleMedianCombiner extends
24 | Reducer {
25 |
26 | private static final Log LOG = LogFactory.getLog(SimpleMedianCombiner.class);
27 | static enum SimpleMedianCombinerStatus { FULL, NOTFULL, MERGED }
28 |
29 | /**
30 | * Reduces values for a given key
31 | * @param key the Key for the given values being passed in
32 | * @param values a List of HolisiticResult objects to combine
33 | * @param context the Context object for the currently executing job
34 | */
35 | public void reduce(LongWritable key, Iterable values,
36 | Context context)
37 | throws IOException, InterruptedException {
38 |
39 |
40 | // now we need to parse the variable dimensions out
41 | int[] variableShape = Utils.getVariableShape( context.getConfiguration());
42 | int[] extractionShape =
43 | Utils.getExtractionShape(context.getConfiguration(),
44 | variableShape.length);
45 | int neededValues = Utils.calcTotalSize(extractionShape);
46 | GroupID tempID = new GroupID();
47 |
48 | HolisticResult holVal = new HolisticResult();
49 | holVal.setNeededValueCount( neededValues );
50 |
51 | for (HolisticResult value : values) {
52 | if ( holVal.isFull() ) {
53 | LOG.warn("Adding an element to an already full HR. Key: " +
54 | key.toString() +
55 | " array size: " + holVal.getNeededValueCount() +
56 | " current elems: " +
57 | holVal.getCurrentValueCount() );
58 | }
59 |
60 | holVal.merge(value);
61 | context.getCounter(SimpleMedianCombinerStatus.MERGED).increment(value.getCurrentValueCount());
62 | }
63 |
64 | // now, the remainig holistic result should be full. Check though
65 | if( holVal.isFull() ) {
66 | // apply whatever function you want, in this case we
67 | // sort and then pull the median out
68 | holVal.sort();
69 | holVal.setFinal( holVal.getValues()[(holVal.getValues().length)/2] );
70 | context.getCounter(SimpleMedianCombinerStatus.FULL).increment(1);
71 | } else {
72 | context.getCounter(SimpleMedianCombinerStatus.NOTFULL).increment(1);
73 | }
74 |
75 | context.write(key, holVal);
76 | }
77 | }
78 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/io/ArraySpec.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.io;
2 |
3 | import java.util.Arrays;
4 | import java.io.IOException;
5 | import java.io.DataInput;
6 | import java.io.DataOutput;
7 |
8 | import org.apache.hadoop.io.WritableComparable;
9 | import org.apache.hadoop.io.Text;
10 |
11 | /**
12 | * This class represents a generic array. It stores
13 | * the data required to open a file and read a contigous
14 | * array shape from a variable in said file.
15 | */
16 | public class ArraySpec implements WritableComparable {
17 | private String _fileName = null;
18 | private String _varName = null;
19 | private int[] _shape = null;
20 | private int[] _corner = null; // anchor point
21 | private int[] _varShape = null; // shape of the entire variabe
22 | private int[] _logicalStartOffset = null; // used to adjust the coordinates
23 | // of the input (in logical space)
24 |
25 | public ArraySpec() {}
26 |
27 | /**
28 | * Constructor
29 | * @param corner The n-dimensional coordinate for the anchoring
30 | * corner of the array to be read
31 | * @param shape The n-dimension shape of the data to be read,
32 | * starting at corner
33 | * @param varName Name of the variable to read the array from
34 | * @param fileName Name of the file to open for reading
35 | * @param variableShape The shape of the variable containing
36 | * this ArraySpec
37 | */
38 | public ArraySpec(int[] corner, int[] shape,
39 | String varName, String fileName,
40 | int[] variableShape)
41 | throws Exception {
42 |
43 | if ( shape.length != corner.length ) {
44 | throw new Exception ("shape and length need to be of the same length");
45 | }
46 |
47 | this._shape = new int[shape.length];
48 | for (int i=0; i < shape.length; i++) {
49 | this._shape[i] = shape[i];
50 | }
51 |
52 | this._corner = new int[corner.length];
53 | for (int i=0; i < corner.length; i++) {
54 | this._corner[i] = corner[i];
55 | }
56 |
57 | this._varShape = new int[variableShape.length];
58 | for( int i=0; i< variableShape.length; i++) {
59 | this._varShape[i] = variableShape[i];
60 | }
61 |
62 | _varName = new String(varName);
63 | _fileName = new String(fileName);
64 | }
65 |
66 | /**
67 | * Constructor where the variable shape is not known
68 | * @param corner The n-dimensional coordinate for the anchoring
69 | * corner of the array to be read
70 | * @param shape The n-dimension shape of the data to be read,
71 | * starting at corner
72 | * @param varName Name of the variable to read the array from
73 | * @param fileName Name of the file to open for reading
74 | */
75 | public ArraySpec(int[] corner, int[] shape,
76 | String varName, String fileName) throws Exception {
77 | this( corner, shape, varName, fileName, new int[0]);
78 | }
79 |
80 | /**
81 | * return the number of dimensions for both shape and corner
82 | * @return the number of dimensions for variable, corner and shape
83 | * (note: one value is returned, all three must have the same number
84 | * of dimensions)
85 | */
86 | public int getRank() {
87 | return this._shape.length;
88 | }
89 |
90 | /**
91 | * Return the corner that anchors the array represented by this ArraySpec
92 | * @return an array of integers representing the coordinate of the corner
93 | * in the respective dimension (array index zero has the coordinate for the
94 | * zero-th dimension, etc.)
95 | */
96 | public int[] getCorner() {
97 | return this._corner;
98 | }
99 |
100 | /**
101 | * Return the shape to be read from the array represented by this ArraySpec
102 | * @return an array of integers representing the length of the shape
103 | * for the respective dimension (array index zero has the length of the
104 | * zero-th dimension, etc.)
105 | */
106 | public int[] getShape() {
107 | return this._shape;
108 | }
109 |
110 | /**
111 | * Return the shape of the n-dimensional variable that contains the
112 | * array represented by this ArraySpec.
113 | * @return an n-dimension array of integers storing the length of
114 | * the variable in the corresponding array location
115 | */
116 | public int[] getVariableShape() {
117 | return this._varShape;
118 | }
119 |
120 | /**
121 | * Get the logical offset for this ArraySpec. This is used to place
122 | * ArraySpecs in logical spaces spanning multiple files where as
123 | * shape and corner and always relative to the specific variable (in the
124 | * specific file) being read.
125 | * @return an n-dimensional array representing the location of this
126 | * ArraySpec in the logical space of the currently executing query
127 | */
128 | public int[] getLogicalStartOffset() {
129 | return this._logicalStartOffset;
130 | }
131 |
132 | /**
133 | * Return the name of the variable containing the data represented by
134 | * this ArraySpec
135 | * @return name of the Variable containing this ArraySpec
136 | */
137 | public String getVarName() {
138 | return _varName;
139 | }
140 |
141 | /**
142 | * Return the name of the file containing the variable which holds
143 | * the data represented by this ArraySpec.
144 | * @return the file name that corresponds to this ArraySpec
145 | */
146 | public String getFileName() {
147 | return _fileName;
148 | }
149 |
150 | /**
151 | * Get the number of cells represented by this ArraySpec.
152 | * @return number of cells represented by this ArraySpec.
153 | */
154 | public long getSize() {
155 | long size = 1;
156 | for (int i = 0; i < this._shape.length; i++) {
157 | size *= this._shape[i];
158 | }
159 |
160 | return size;
161 | }
162 |
163 | /**
164 | * Set the shape of the data to be read
165 | * @param newShape shape of the data to be read
166 | */
167 | public void setShape( int[] newShape ) {
168 | // might want to do some checking of old shape vs new shape later
169 | this._shape = newShape;
170 | }
171 |
172 | /**
173 | * Set the shape of the variable that contains the data represented
174 | * by this ArraySpec.
175 | * @param newVarShape the Shape of the variable that contains the
176 | * data for this ArraySpec
177 | */
178 | public void setVariableShape( int[] newVarShape) {
179 | this._varShape = newVarShape;
180 | }
181 |
182 | /**
183 | * Sets the logical offset of the this ArraySpec
184 | * @param newLogicalStartOffset the offset, in the global logical
185 | * space, where this ArraySpec resides
186 | */
187 | public void setLogicalStartOffset( int[] newLogicalStartOffset ){
188 | this._logicalStartOffset = newLogicalStartOffset;
189 | }
190 |
191 | /**
192 | * Write the contents of this ArraySpec out to a string
193 | * @return a String representation of this object
194 | */
195 | public String toString() {
196 | return _fileName + ": var: " + _varName + ": corner = " +
197 | Arrays.toString(_corner) +
198 | ", shape = " + Arrays.toString(_shape);
199 | }
200 |
201 | /**
202 | * Compares the current ArraySpec to another ArraySpec
203 | * @return an integer that is less than, equal to, or greater than
204 | * zero depending on whether the object passed in is less than,
205 | * equal to or greater than this object, respectively
206 | */
207 | public int compareTo(Object o) {
208 | int retVal = 0;
209 | ArraySpec other = (ArraySpec)o;
210 |
211 | if ( 0 != this._fileName.compareTo(other.getFileName())){
212 | return this._fileName.compareTo(other.getFileName());
213 | }
214 |
215 | if ( 0 != this._varName.compareTo(other.getVarName())){
216 | return this._varName.compareTo(other.getVarName());
217 | }
218 |
219 | for ( int i = 0; i < this._corner.length; i++) {
220 | retVal = this._corner[i] - other.getCorner()[i];
221 |
222 | if (retVal != 0) {
223 | return retVal;
224 | }
225 | }
226 |
227 | return retVal;
228 | }
229 |
230 | /**
231 | * Serialize the contents of this ArraySpec to a DataOutput object
232 | * @param out The DataOutput object to write the contents of this
233 | * ArraySpec to
234 | */
235 | @Override
236 | public void write(DataOutput out) throws IOException {
237 | Text.writeString(out, _fileName);
238 | Text.writeString(out, _varName);
239 |
240 | out.writeInt(_shape.length);
241 | for (int i = 0; i < _shape.length; i++)
242 | out.writeInt(_shape[i]);
243 |
244 | out.writeInt(_corner.length);
245 | for (int i = 0; i < _corner.length; i++)
246 | out.writeInt(_corner[i]);
247 |
248 | out.writeInt(_varShape.length);
249 | for (int i = 0; i < _varShape.length; i++)
250 | out.writeInt(_varShape[i]);
251 |
252 | if ( null == _logicalStartOffset ) {
253 | out.writeInt(0);
254 | } else {
255 | out.writeInt(_logicalStartOffset.length);
256 | for (int i= 0; i < _logicalStartOffset.length; i++) {
257 | out.writeInt(_logicalStartOffset[i]);
258 | }
259 | }
260 | }
261 |
262 | /**
263 | * Populate an ArraySpec object by reading data from a
264 | * DataInput object
265 | * @param in The DataInput object to read the data from
266 | */
267 | @Override
268 | public void readFields(DataInput in) throws IOException {
269 | _fileName = Text.readString(in);
270 | _varName = Text.readString(in);
271 |
272 | int len = in.readInt();
273 | _shape = new int[len];
274 | for (int i = 0; i < _shape.length; i++)
275 | _shape[i] = in.readInt();
276 |
277 | len = in.readInt();
278 | _corner = new int[len];
279 | for (int i = 0; i < _corner.length; i++)
280 | _corner[i] = in.readInt();
281 |
282 | len = in.readInt();
283 | _varShape = new int[len];
284 | for (int i = 0; i < _varShape.length; i++)
285 | _varShape[i] = in.readInt();
286 |
287 | len = in.readInt();
288 | if ( 0 == len ) {
289 | _logicalStartOffset = null;
290 | } else {
291 | _logicalStartOffset = new int[len];
292 | for (int i = 0; i < _logicalStartOffset.length; i++)
293 | _logicalStartOffset[i] = in.readInt();
294 | }
295 | }
296 | }
297 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/io/AverageResult.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.io;
2 |
3 | import java.util.Arrays;
4 | import java.io.IOException;
5 | import java.io.DataInput;
6 | import java.io.DataOutput;
7 |
8 | import edu.ucsc.srl.damasc.netcdf.Utils;
9 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
10 |
11 | import org.apache.commons.logging.Log;
12 | import org.apache.commons.logging.LogFactory;
13 |
14 | import org.apache.hadoop.io.Writable;
15 | import org.apache.hadoop.io.Text;
16 |
17 | /**
18 | * This class represents the results of an average operation.
19 | * It tracks the results of average for part of the data and allows
20 | * for accurrate aggregation of partial results into the final result.
21 | */
22 | public class AverageResult implements Writable {
23 | private double _currentValue;
24 | private int _valuesCombinedCount;
25 |
26 | private static final Log LOG = LogFactory.getLog(AverageResult.class);
27 |
28 | /**
29 | * Constructor
30 | */
31 | public AverageResult() {
32 | // this._currentValue = 0;
33 | //this._valuesCombinedCount = 0;
34 | this.clear();
35 | }
36 |
37 |
38 | /**
39 | * Create an AverageResult object based on
40 | * a single value. the mapper uses this to do fast creation
41 | * @param value the single value to use to seed this AverageResult
42 | * object
43 | */
44 | public AverageResult(int value ) throws Exception {
45 | this._currentValue = (double)value;
46 | this._valuesCombinedCount = 1;
47 | }
48 |
49 | /**
50 | * Constructor that takes both a current value and a count of the values
51 | * that were combined to get that value.
52 | * @param value the current average value of the results aggregated
53 | * @param valudsCombinedCount the number of values that have been
54 | * combined so far
55 | */
56 | public AverageResult(int value, int valuesCombinedCount) throws Exception {
57 | this._currentValue = (double)value;
58 | this._valuesCombinedCount = valuesCombinedCount;
59 | }
60 |
61 | /**
62 | * Re-initialize this object for potential reuse
63 | */
64 | public void clear() {
65 | this._currentValue = (double)0;
66 | this._valuesCombinedCount = 0;
67 | }
68 |
69 | /**
70 | * Return the current average based on the values that have been
71 | * processed so far
72 | * @return the current average value
73 | */
74 | public int getCurrentValue() {
75 | return (int)this._currentValue;
76 | }
77 |
78 | /**
79 | * Return the current average value as a double
80 | * @return the current average value as a double
81 | */
82 | public double getCurrentValueDouble() {
83 | return this._currentValue;
84 | }
85 |
86 | /**
87 | * The number of values that have been added into the running
88 | * average so far
89 | * @return the count of values that have been combined so far
90 | */
91 | public int getValuesCombinedCount() {
92 | return this._valuesCombinedCount;
93 | }
94 |
95 | /**
96 | * Adds a value to the running total. This will update both
97 | * the average value and the count of values that have been
98 | * processed so far
99 | * @param value the value to add to the running average
100 | */
101 | public void addValue(int value) throws IOException {
102 | //LOG.info("pre-add (int): value: " + this._currentValue + " count: " + this._valuesCombinedCount);
103 | this._currentValue = ((this._currentValue * this._valuesCombinedCount) + value ) / (this._valuesCombinedCount + 1);
104 | this._valuesCombinedCount++;
105 | //LOG.info("post-add (int): value: " + this._currentValue + " count: " + this._valuesCombinedCount);
106 | }
107 |
108 | /**
109 | * Merge this AverageResult object with another
110 | * @param result the AverageResult object to merge with this one
111 | */
112 | public void addAverageResult( AverageResult result ) {
113 | //LOG.info("pre-add (AvgRes): value: " + this._currentValue + " count: " + this._valuesCombinedCount +
114 | // " in.value: " + result.getCurrentValueDouble() + " in.count: " + result.getValuesCombinedCount() );
115 | this._currentValue = ((this._currentValue * this._valuesCombinedCount) +
116 | (result.getCurrentValueDouble() * result.getValuesCombinedCount()))
117 | / ((double)this._valuesCombinedCount + result.getValuesCombinedCount());
118 | this._valuesCombinedCount += result.getValuesCombinedCount();
119 | //LOG.info("post-add (AvgRes): value: " + this._currentValue + " count: " + this._valuesCombinedCount +
120 | // " in.value: " + result.getCurrentValueDouble() + " in.count: " + result.getValuesCombinedCount() );
121 |
122 | }
123 |
124 | /**
125 | * Set the current average value from an int
126 | * @param value the value to set as the current average value.
127 | */
128 | protected void setCurrentValue( int value ) {
129 | this._currentValue = (double)value;
130 | }
131 |
132 | /**
133 | * Set the current value from a double
134 | * @param value the double value to use to set the current value
135 | */
136 | protected void setCurrentValue( double value ) {
137 | this._currentValue = value;
138 | }
139 |
140 | /**
141 | * Set the number of values combined to form the current
142 | * average value
143 | * @param newCount the new number of values used to
144 | * calculate the current average value
145 | */
146 | public void setValuesCombinedCount( int newCount ) {
147 | this._valuesCombinedCount = newCount;
148 | }
149 |
150 | /**
151 | * Set both the current average value and the count
152 | * of the number of values processed to get to that average.
153 | * @param value the current average value
154 | * @param newCount how many values have been processed so far
155 | */
156 | protected void setValue( int value, int newCount) {
157 | this.setCurrentValue( value);
158 | this.setValuesCombinedCount( newCount);
159 | }
160 |
161 | /**
162 | * Write the contents of this AverageResult object out to a String
163 | * @return a String representation of this AverageResult object
164 | */
165 | public String toString() {
166 | return "value = " + this._currentValue +
167 | " count = " + this._valuesCombinedCount;
168 | }
169 |
170 | /**
171 | * Serialize this AverageResult object to a DataOutput object
172 | * @param out the DataOutput object to write the contents of this
173 | * object to
174 | */
175 | @Override
176 | public void write(DataOutput out) throws IOException {
177 | out.writeDouble(this._currentValue);
178 | out.writeInt(this._valuesCombinedCount);
179 | }
180 |
181 | /**
182 | * Populate this AverageResult object from the data read from
183 | * a DataInput object
184 | * @param in the DataInput object to read data from
185 | */
186 | @Override
187 | public void readFields(DataInput in) throws IOException {
188 |
189 | this.setCurrentValue(in.readDouble());
190 | this.setValuesCombinedCount(in.readInt());
191 | }
192 | }
193 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/io/GroupID.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.io;
2 |
3 | import java.util.Arrays;
4 | import java.io.IOException;
5 | import java.io.DataInput;
6 | import java.io.DataOutput;
7 |
8 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
9 | import edu.ucsc.srl.damasc.netcdf.Utils;
10 |
11 | import org.apache.commons.logging.Log;
12 | import org.apache.commons.logging.LogFactory;
13 |
14 | import org.apache.hadoop.io.WritableComparable;
15 | import org.apache.hadoop.io.Text;
16 |
17 | /**
18 | * Stores the instance of the extraction shape that corresponds to an
19 | * array. Serves as a key for the shuffle phase of MapReduce jobs.
20 | */
21 | public class GroupID implements WritableComparable {
22 | private String _name; // the variable name this data came from
23 | private int[] _groupID; // the corner of a given extraction shape
24 |
25 | public GroupID() {}
26 |
27 | private static final Log LOG = LogFactory.getLog(GroupID.class);
28 |
29 |
30 | /**
31 | * Constructor that sets the GroupID and the name of the variable
32 | * the ID corresponds to
33 | * @param groupID GroupID, as an n-dimensional variable
34 | * @param name Variable name that this GroupID belongs to
35 | */
36 | public GroupID(int[] groupID, String name) throws Exception {
37 |
38 | this._groupID = new int[groupID.length];
39 | for (int i=0; i < groupID.length; i++) {
40 | this._groupID[i] = groupID[i];
41 | }
42 |
43 | this._name = new String(name);
44 | }
45 |
46 | /**
47 | * Return the number of dimensions for this GroupID
48 | * @return the number of dimensions for the variable
49 | * that this GroupID belongs to
50 | */
51 | public int getRank() {
52 | return this._groupID.length;
53 | }
54 |
55 | /**
56 | * Returns the GroupID
57 | * @return The GroupID as an n-dimensional array
58 | */
59 | public int[] getGroupID() {
60 | return this._groupID;
61 | }
62 |
63 | /**
64 | * Returns the name of the variable that this GroupID corresponds to
65 | * @return variable name as a String
66 | */
67 | public String getName() {
68 | return this._name;
69 | }
70 |
71 | /**
72 | * Sets the group ID for this GroupID object
73 | * @param newGroupID the ID for this object
74 | */
75 | public void setGroupID( int[] newGroupID) {
76 | this._groupID = newGroupID;
77 | }
78 |
79 | /**
80 | * Makes it possible to set the ID for a specific dimension
81 | * @param dim The dimension to set
82 | * @param val The value to set indicated dimension to
83 | */
84 | public void setDimension( int dim, int val)
85 | throws ArrayIndexOutOfBoundsException {
86 | if ( dim < 0 || dim >= this._groupID.length)
87 | throw new ArrayIndexOutOfBoundsException("setDimension called with " +
88 | "dimension " + dim + " on groupID with dimensions " +
89 | this._groupID.length);
90 |
91 | this._groupID[dim] = val;
92 | }
93 |
94 | /**
95 | * Sets the variable name for this GroupID object
96 | * @param newName the name of the Variable for this object
97 | */
98 | public void setName( String newName ) {
99 | this._name = newName;
100 | }
101 |
102 | /**
103 | * Returns the contents of this GroupID object as a String
104 | * @return a String version of this GroupID object
105 | */
106 | public String toString() {
107 | return _name + ": groupID = " + Arrays.toString(this._groupID);
108 | }
109 |
110 | /**
111 | * Projects this GroupID from the local logical space
112 | * into the global logical space via the extraction shape
113 | * @param exShape The extraction shape to use to project this
114 | * GroupID into the global space
115 | * @return the group ID for this object, in the global logical space
116 | */
117 | public String toString(int[] exShape) {
118 | int[] tempArray = new int[this._groupID.length];
119 | for ( int i=0; i 1 &&
168 | (this._values.length == this._currentValueCount) ) {
169 | this._full = true;
170 | }
171 |
172 | return this._full;
173 | }
174 |
175 | /**
176 | * Adds a value to this result object
177 | * @param value the value to add to this object
178 | */
179 | public void setValue(int value) throws IOException {
180 | if( this._values.length > 1 && this.isFull()) {
181 | throw new IOException("ERROR: adding an element to an already " +
182 | "full HolisticResult object." +
183 | "Length: " + this._values.length);
184 | }
185 |
186 | if ( this._final == true ) {
187 | throw new IOException("ERROR: adding a value to a " +
188 | "HolisticResult that has been marked final");
189 | }
190 |
191 | this._values[this._currentValueCount] = value;
192 | this._currentValueCount++;
193 | }
194 |
195 | /**
196 | * This means that the result for this object
197 | * has been calculated. This generates a new array, holding only the
198 | * result, and sets the "final" status to true.
199 | * @param value the result for this result object
200 | */
201 | public void setFinal( int value ) throws IOException {
202 | this._values = new int[1]; // free up the now useless ._values array
203 | this._currentValueCount = 0;
204 | this.setValue(value);
205 | this._final = true;
206 | }
207 |
208 |
209 | /**
210 | * Returns the contents of this result object in String form
211 | * @return the contents of this object as a String
212 | */
213 | public String toString() {
214 | return "values = " + Utils.arrayToString(this._values);
215 | }
216 |
217 | /**
218 | * Initializes this object to having a single value in it.
219 | * This is used to reset a result object
220 | * @param value the single value to seed this result object with
221 | */
222 | public void setHolisticResult( int value ) throws IOException {
223 | this._values = new int[1];
224 | this._currentValueCount = 0;
225 | this.setValue(value);
226 | }
227 |
228 | /**
229 | * Initializes a result object with an array of results to
230 | * add in to this object.
231 | * TODO: optimize this by allocating _values here
232 | * @param values the array of values to add to this object
233 | */
234 | public void setHolisticResult( int[] values ) throws IOException {
235 |
236 | for ( int i=0; i records = new ArrayList();
236 |
237 | // this next bit is fairly hard-coded and specific to our tests.
238 | // it represents a naive split that a human might come up with
239 |
240 | // sort out the step size
241 | int[] stepShape = calcStepShape(dims, blockSize, numBlocks,
242 | fileLen, dataTypeSize, conf);
243 |
244 | int[] tempCorner = new int[ndims];
245 | int[] tempStep = new int[ndims];
246 |
247 | // initialize the temporary step shape to be the first step
248 | for( int i=0; i> blockToArrays,
325 | long totalArraySpecCount, String fileName,
326 | Configuration conf )
327 | throws IOException {
328 |
329 | // place each ArraySpec in the correct block
330 | for (ArraySpec record : records) {
331 | int blockToInsertTo =
332 | (int) (totalArraySpecCount % blockToArrays.size());
333 | /*
334 | System.out.println("record corner: " +
335 | Utils.arrayToString(record.getCorner() ) +
336 | " going into block " +
337 | blockToInsertTo + " which starts at offset " +
338 | blocks[blockToInsertTo] );
339 | */
340 |
341 | blockToArrays.get( blocks[blockToInsertTo] ).add(record);
342 | totalArraySpecCount++;
343 | }
344 |
345 | return totalArraySpecCount;
346 | }
347 |
348 | /*
349 | public List orderMultiFileInput( List files/
350 | Configuration conf ) {
351 | List retList = new ArrayList();
352 |
353 | // first, sort the files in alphanumeric order
354 | Collections.sort(files);
355 |
356 | int[] startOffset = null;
357 | // now go through them, in order
358 | for (FileStatus file: files) {
359 |
360 | Variable var = getVariable(file, conf);
361 |
362 | if ( startOffset == null ){
363 | startOffset = new int[var.getDimensions().size()];
364 | for( int i=0; i getSplits(JobContext job) throws IOException {
393 | List splits = new ArrayList();
394 | //List files = listStatus(job);
395 |
396 | // HashMap> blockToSlab =
397 | // new HashMap>();
398 |
399 | /*
400 | FileStatus ncfileStatus = null;
401 |
402 |
403 | for (FileStatus file: files) {
404 | if (ncfileStatus == null) {
405 | ncfileStatus = file;
406 | LOG.info("Using input: " + file.getPath().toString());
407 | } else {
408 | LOG.warn("Skipping input: " + file.getPath().toString());
409 | }
410 | }
411 |
412 | if (ncfileStatus == null)
413 | return splits;
414 | */
415 | /*
416 | PartMode partMode = Utils.getPartMode(job.getConfiguration());
417 | PlacementMode placementMode =
418 | Utils.getPlacementMode(job.getConfiguration());
419 | */
420 |
421 | /*
422 | if (Utils.getMultiFileMode(job.getConfiguration()) == MultiFileMode.concat) {
423 | orderMultiFileInput( files, shFiles);
424 | }
425 | */
426 |
427 | // set the starting offset for each file (depends on damasc.multi_file_mode
428 | /*
429 | shFiles = orderMultiFileInput( files, job.getConfiguration() );
430 |
431 | for (SHFileStatus shFile: shFiles) {
432 | LOG.info("Parsing file: " + shFile.getFileStatus().getPath().toString());
433 | Utils.addFileName(shFile.getFileStatus().getPath().toString(), job.getConfiguration());
434 | genFileSplits(job, shFile, splits, partMode, placementMode);
435 | }
436 |
437 |
438 | // debug: log splits to a file if the debug log files is set
439 | String debugFileName = Utils.getDebugLogFileName(job.getConfiguration());
440 | if ( "" != debugFileName ) {
441 | LOG.info("Trying to log to " + debugFileName);
442 | File outputFile = new File( debugFileName );
443 | BufferedWriter writer = new BufferedWriter( new FileWriter(outputFile));
444 |
445 | int i = 0;
446 | for (InputSplit split : splits) {
447 | ArrayBasedFileSplit tempSplit = (ArrayBasedFileSplit)split;
448 | //LOG.info("Split " + i);
449 | writer.write("Splits " + i);
450 | writer.newLine();
451 | for ( ArraySpec spec : tempSplit.getArraySpecList() ) {
452 | writer.write("File: " + spec.getFileName() +
453 | "\tvar: " + spec.getVarName() +
454 | "\tcorner: " + Utils.arrayToString( spec.getCorner()) +
455 | "\t shape: " + Utils.arrayToString( spec.getShape() ) +
456 | "\t startOffset: " + Utils.arrayToString( spec.getLogicalStartOffset()) );
457 | writer.newLine();
458 | }
459 | i++;
460 | }
461 | writer.close();
462 | } else {
463 | LOG.info("No debugFileName set");
464 | }
465 | */
466 |
467 | return splits;
468 | }
469 | }
470 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/io/input/ArrayBasedFileSplit.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.io.input;
2 |
3 | import java.io.IOException;
4 | import java.io.DataInput;
5 | import java.io.DataOutput;
6 | import java.util.ArrayList;
7 |
8 | import org.apache.commons.logging.Log;
9 | import org.apache.commons.logging.LogFactory;
10 | import org.apache.hadoop.mapreduce.InputSplit;
11 | import org.apache.hadoop.io.Writable;
12 | import org.apache.hadoop.io.Text;
13 | import org.apache.hadoop.fs.Path;
14 |
15 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
16 |
17 | /**
18 | * This class is meant to represent splits for Array-based files.
19 | */
20 | public class ArrayBasedFileSplit extends InputSplit implements Writable {
21 |
22 | private Path _path = null; // path of the in HDFS
23 |
24 | // Array of host names that this split *should* be assigned to
25 | private String[] _hosts = null;
26 |
27 | private long _totalDataElements = 0; // total number of cells represented by this split
28 | private ArrayList _arraySpecList; // List of ArraySpecs for this split
29 |
30 | private static final Log LOG = LogFactory.getLog(ArrayBasedFileSplit.class);
31 |
32 | public ArrayBasedFileSplit() {}
33 |
34 | /**
35 | * NOTE: start and shape must be the same length
36 | *
37 | * Constructor for an ArrayBasedFileSplit
38 | * @param file the file name
39 | * @param variable the name of the variable this data belongs to
40 | * @param shape an n-dimensional array representing the length, in each dimension, of this array
41 | * @param dimToIterateOver which dimension to iterate over
42 | * @param startStep which step in the dimension being iterated over to start on
43 | * @param numSteps how many steps this split represents
44 | * @param hosts the list of hosts containing this block, possibly null
45 | */
46 | public ArrayBasedFileSplit( Path path, ArrayList arraySpecList,
47 | String[] hosts) {
48 | this._path = new Path(path.toString());
49 | this._arraySpecList = new ArrayList(arraySpecList);
50 |
51 | this._hosts = new String[hosts.length];
52 | for (int i=0; i embedded within this split
71 | * @return ArrayList with specific ArraySpecs
72 | */
73 | public ArrayList getArraySpecList() {
74 | return this._arraySpecList;
75 | }
76 |
77 | @Override
78 | /**
79 | * Returns the total number of cells represented by the group of
80 | * ArraySpec entries in this split
81 | * @return total number of cells represented by this split
82 | */
83 | public long getLength() throws IOException, InterruptedException {
84 | return this._totalDataElements;
85 | }
86 |
87 | @Override
88 | /**
89 | * Returns a list of hosts that locally possess a copy of the file system
90 | * block that this Split corresponds to
91 | * @return array of hostnames as Strings
92 | */
93 | public String[] getLocations() throws IOException, InterruptedException {
94 | if (null == this._hosts)
95 | return new String[]{};
96 | else
97 | return this._hosts;
98 | }
99 |
100 | /**
101 | * Serializes this Split to the DataOutput object
102 | * @param out DataOutput object to serialize this Split to
103 | */
104 | public void write(DataOutput out) throws IOException {
105 | Text.writeString(out, this._path.toString());
106 |
107 | // first, write the number of entires in the list
108 | out.writeInt(this._arraySpecList.size() );
109 |
110 | // then, for each entry, write it
111 | for (int i=0; i(listSize);
127 |
128 | // for each element, create a corner and shape
129 | for (int i=0; i< listSize; i++) {
130 | ArraySpec array = new ArraySpec();
131 | array.readFields(in);
132 | this._arraySpecList.add(array);
133 | }
134 | }
135 |
136 | /**
137 | * Compares this object to another and returns a negative value,
138 | * zero or a positive value if this object is less than, equal to
139 | * or greater than the object passed in.
140 | * @param o the Object to compare this Split to. Assumed to bed another
141 | * ArrayBasedFileSplit
142 | * @return a negative value, zero or positive value if this object
143 | * is less than, equal to or greater than, respectively, the object
144 | * passed in
145 | */
146 | public int compareTo(Object o) {
147 | ArrayBasedFileSplit temp = (ArrayBasedFileSplit)o;
148 | int retVal = 0;
149 |
150 | // we'll compare the first entry in the ArrayList
151 | // from each object. If they are
152 | // the same, then we'll assume these are the same element
153 | // (since each ArraySpec should exist in one and
154 | // only one entry
155 | ArraySpec arrayA = this._arraySpecList.get(0);
156 | ArraySpec arrayB = temp.getArraySpecList().get(0);
157 |
158 | if (arrayA == null) {
159 | return -1;
160 | } else if ( arrayB == null) {
161 | return 1;
162 | }
163 |
164 | for (int i=0; i < arrayA.getCorner().length; i++) {
165 | retVal = new Integer(arrayA.getCorner()[i] -
166 | arrayB.getCorner()[i]).intValue();
167 | if (retVal != 0) {
168 | return retVal;
169 | }
170 | }
171 |
172 | return 0;
173 | }
174 |
175 | /**
176 | * Handy function to print out the contents of this split
177 | * @return string versions of the data contained in this split
178 | */
179 | public String toString() {
180 | String tempStr = "";
181 |
182 | tempStr += "file: " + this._path.getName() + " with ArraySpecs:\n";
183 |
184 | for ( int i=0; i _arraySpecList;
27 | private static final Log LOG = LogFactory.getLog(NetCDFFileSplit.class);
28 |
29 | public NetCDFFileSplit() {}
30 |
31 | /**
32 | * NOTE: start and shape must be the same length
33 | *
34 | * @param file the file name
35 | * @param variable the name of the variable this data belongs to
36 | * @param shape an n-dimensional array representing the length,
37 | * in each dimension, of this array
38 | * @param dimToIterateOver which dimension to iterate over
39 | * @param startStep which step in the dimension being iterated over to start on
40 | * @param numSteps how many steps this split represents
41 | * @param hosts the list of hosts containing this block, possibly null
42 | */
43 | public NetCDFFileSplit( Path path, ArrayList arraySpecList,
44 | String[] hosts) {
45 | this._path = new Path(path.toString());
46 | this._arraySpecList = new ArrayList(arraySpecList);
47 |
48 | this._hosts = new String[hosts.length];
49 | for (int i=0; i getArraySpecList() {
72 | return this._arraySpecList;
73 | }
74 |
75 | /**
76 | * Returns the number of cells represented by the set of ArraySpecs
77 | * in this Split
78 | * @return the number of total cells represented by this split
79 | */
80 | @Override
81 | public long getLength() throws IOException, InterruptedException {
82 | return this._totalDataElements;
83 | }
84 |
85 | /**
86 | * Returns the list of hosts which contain, locally, the file system
87 | * block that this InputSplit is assigned to
88 | * @return an array of Strings that are hostnames. The scheduler should attempt
89 | * to place this InputSplit on one of those hosts
90 | */
91 | @Override
92 | public String[] getLocations() throws IOException, InterruptedException {
93 | if (null == this._hosts)
94 | return new String[]{};
95 | else
96 | return this._hosts;
97 | }
98 |
99 | /**
100 | * Serialze this structure to a DataOutput object
101 | * @param out The DataOutput object to write the context of this split to
102 | */
103 | public void write(DataOutput out) throws IOException {
104 | Text.writeString(out, this._path.toString());
105 |
106 | // first, write the number of entires in the list
107 | out.writeInt(this._arraySpecList.size() );
108 |
109 | // then, for each entry, write it
110 | for (int i=0; i(listSize);
126 |
127 | // for each element, create a corner and shape
128 | for (int i=0; i< listSize; i++) {
129 | ArraySpec array = new ArraySpec();
130 | array.readFields(in);
131 | this._arraySpecList.add(array);
132 | }
133 | }
134 |
135 | /**
136 | * Compares this NetCDFFileSplit to another.
137 | * @return an int that is less than, equal to, or greater than 0,
138 | * depending on whether the object passed in is lesser than, equal to,
139 | * or greater than this NetCDFFileSplit (respectively)
140 | */
141 | public int compareTo(Object o) {
142 | NetCDFFileSplit temp = (NetCDFFileSplit)o;
143 | int retVal = 0;
144 |
145 | // we'll compare the first entry in the ArrayList
146 | // from each object. If they are
147 | // the same, then we'll assume these are the same element
148 | // (since each ArraySpec should exist in one and
149 | // only one entry
150 | ArraySpec arrayA = this._arraySpecList.get(0);
151 | ArraySpec arrayB = temp.getArraySpecList().get(0);
152 |
153 | if (arrayA == null) {
154 | return -1;
155 | } else if ( arrayB == null) {
156 | return 1;
157 | }
158 |
159 | for (int i=0; i < arrayA.getCorner().length; i++) {
160 | retVal = new Integer(arrayA.getCorner()[i] -
161 | arrayB.getCorner()[i]).intValue();
162 | if (retVal != 0) {
163 | return retVal;
164 | }
165 | }
166 |
167 | return 0;
168 | }
169 |
170 | /**
171 | * Writes the contents of this NetCDFFileSplit to a string
172 | * @return a String representing the content of this NetCDFFileSplit
173 | */
174 | public String toString() {
175 | String tempStr = "";
176 |
177 | tempStr += "file: " + this._path.getName() + " with ArraySpecs:\n";
178 |
179 | for ( int i=0; i {
42 |
43 | private static final Log LOG = LogFactory.getLog(NetCDFRecordReader.class);
44 | private long _timer;
45 | private int _numArraySpecs;
46 |
47 | //this will cause the library to use its default size
48 | private int _bufferSize = -1;
49 |
50 | private NetcdfFile _ncfile = null;
51 | private NcHdfsRaf _raf = null;
52 | private Variable _curVar; // actual Variable object
53 | private String _curVarName; // name of the current variable that is open
54 | private String _curFileName;
55 |
56 | // how many data elements were read the last step
57 | private long _totalDataElements = 1;
58 |
59 | // how many data elements have been read so far (used to track work done)
60 | private long _elementsSeenSoFar = 0;
61 |
62 | private ArrayList _arraySpecArrayList = null;
63 |
64 | private ArraySpec _currentArraySpec = null; // this also serves as key
65 | private Array _value = null;
66 | private int _currentArraySpecIndex = 0;
67 |
68 | /**
69 | * Resets a RecordReader each time it is passed a new InputSplit to read
70 | * @param genericSplit an InputSplit (really an ArrayBasedFileSplit) that
71 | * needs its data read
72 | * @param context TaskAttemptContext for the currently executing progrma
73 | */
74 | @Override
75 | public void initialize(InputSplit genericSplit, TaskAttemptContext context)
76 | throws IOException {
77 |
78 | this._timer = System.currentTimeMillis();
79 | ArrayBasedFileSplit split = (ArrayBasedFileSplit)genericSplit;
80 | this._numArraySpecs = split.getArraySpecList().size();
81 | Configuration job = context.getConfiguration();
82 |
83 | Path path = split.getPath();
84 | FileSystem fs = path.getFileSystem(job);
85 |
86 | this._arraySpecArrayList = split.getArraySpecList();
87 |
88 | // calculate the total data elements in this split
89 | this._totalDataElements = 0;
90 |
91 | for ( int j=0; j < this._arraySpecArrayList.size(); j++) {
92 | this._totalDataElements += this._arraySpecArrayList.get(j).getSize();
93 | }
94 |
95 | // get the buffer size
96 | this._bufferSize = Utils.getBufferSize(job);
97 |
98 | this._raf = new NcHdfsRaf(fs.getFileStatus(path), job, this._bufferSize);
99 | this._ncfile = NetcdfFile.open(this._raf, path.toString());
100 |
101 | }
102 |
103 | /** this is called to load the next key/value in. The actual data is retrieved
104 | * via getCurrent[Key|Value] calls
105 | * @return a boolean that is true if there is more data to be read,
106 | * false otherwise
107 | */
108 | @Override
109 | public boolean nextKeyValue() throws IOException, InterruptedException {
110 | if ( !this._arraySpecArrayList.isEmpty() ) {
111 | // set the current element
112 | this._currentArraySpec = this._arraySpecArrayList.get(0);
113 |
114 | // then delete it from the ArrayList
115 | this._arraySpecArrayList.remove(0);
116 |
117 | // fixing an entirely random bug -jbuck TODO FIXME
118 | if ( this._currentArraySpec.getCorner().length <= 1 ) {
119 | return this.nextKeyValue();
120 | }
121 |
122 | // transfer the data
123 | loadDataFromFile();
124 |
125 | return true;
126 | } else {
127 | this._timer = System.currentTimeMillis() - this._timer;
128 | LOG.debug("from init() to nextKeyValue() returning false, " +
129 | "this record reader took: " + this._timer +
130 | " ms. It had " + this._numArraySpecs +
131 | " ArraySpecs to process" );
132 |
133 | return false;
134 | }
135 | }
136 |
137 | /**
138 | * Load data into the value element from disk.
139 | * Currently this only supports IntWritable. Extend this
140 | * to support other data types TODO
141 | */
142 | private void loadDataFromFile() throws IOException {
143 | try {
144 |
145 | // reuse the open variable if it's the correct one
146 | if ( this._curVarName == null ||
147 | 0 != (this._currentArraySpec.getVarName()).compareTo(this._curVarName)){
148 | LOG.debug("calling getVar on " + this._currentArraySpec.getVarName() );
149 | this._curVar =
150 | this._ncfile.findVariable(this._currentArraySpec.getVarName());
151 | }
152 |
153 |
154 | if ( this._curVar ==null ) {
155 | LOG.warn("this._curVar is null. BAD NEWS");
156 | LOG.warn( "file: " + this._currentArraySpec.getFileName() +
157 | "corner: " +
158 | Arrays.toString(this._currentArraySpec.getCorner() ) +
159 | " shape: " + Arrays.toString(this._currentArraySpec.getShape() ) );
160 | }
161 |
162 | LOG.warn( " File: " + this._currentArraySpec.getFileName() +
163 | " startOffset: " + Utils.arrayToString(this._currentArraySpec.getLogicalStartOffset()) +
164 | "corner: " +
165 | Arrays.toString(this._currentArraySpec.getCorner()) +
166 | " shape: " +
167 | Arrays.toString(this._currentArraySpec.getShape()));
168 |
169 | // this next bit is to be able to set the dimensions of the variable
170 | // for this ArraySpec. Needed for flattening the groupID to a long
171 | ArrayList varDims =
172 | new ArrayList(this._curVar.getDimensions());
173 | int[] varDimLengths = new int[varDims.size()];
174 |
175 | for( int i=0; i {
30 |
31 | private static final Log LOG = LogFactory.getLog(AverageMapper.class);
32 | private static boolean _benchmarkArraySpec = true;
33 |
34 | public static enum InvalidCell { INVALID_CELL_COUNT } ;
35 |
36 | /**
37 | * Reduces values for a given key
38 | * @param key the Key for the given value being passed in
39 | * @param value an Array to process that corresponds to the given key
40 | * @param context the Context object for the currently executing job
41 | */
42 | public void map(ArraySpec key, Array value, Context context)
43 | throws IOException, InterruptedException {
44 | try {
45 |
46 | long timerA = System.currentTimeMillis();
47 |
48 | ArrayInt ncArray = (ArrayInt)value;
49 |
50 | int[] allOnes = new int[key.getShape().length];
51 | for( int i=0; i groupSubArrayMap = new HashMap();
62 |
63 | GroupIDGen myGIDG = new GroupIDGen();
64 | GroupIDGen.pullOutSubArrays( myGIDG, ncArray, key, extractionShape,
65 | allOnes, groupSubArrayMap);
66 | LOG.info("pullOutSubArrays returned " + groupSubArrayMap.size() + " elements");
67 |
68 | ArrayInt localArray;
69 | GroupID localGID = new GroupID();
70 | localGID.setName(key.getVarName());
71 | int tempInt = 0;
72 |
73 | int[] zeroArray = new int[extractionShape.length];
74 | for( int i = 0; i < zeroArray.length; i++) {
75 | zeroArray[i] = 0;
76 | }
77 |
78 | int[] helpArray = new int[extractionShape.length];
79 | for( int i = 0; i < helpArray.length; i++) {
80 | helpArray[i] = 0;
81 | }
82 |
83 | GroupID zeroGID = new GroupID(zeroArray, "windspeed1");
84 |
85 | int invalidMapperCounter = 0;
86 |
87 | LongWritable myLongW = new LongWritable();
88 | AverageResult avgResult = new AverageResult();
89 |
90 | // debugging bit here
91 | Iterator> gidItr2 =
92 | groupSubArrayMap.entrySet().iterator();
93 |
94 | System.out.println("ArraySpec corner: " + Utils.arrayToString(key.getCorner()) +
95 | " shape: " + Utils.arrayToString(key.getShape()));
96 | while (gidItr2.hasNext() ) {
97 | System.out.println("gid: " + Utils.arrayToString(gidItr2.next().getKey().getGroupID()));
98 | }
99 |
100 | Iterator> gidItr =
101 | groupSubArrayMap.entrySet().iterator();
102 |
103 | while (gidItr.hasNext() ) {
104 | Map.Entry pairs = gidItr.next();
105 | localGID = pairs.getKey();
106 | localArray = (ArrayInt)pairs.getValue();
107 | avgResult.clear(); // reset this variable
108 |
109 | // TODO sort out how to do filtering with this new GroupID based setup
110 | // -jbuck
111 |
112 | IndexIterator valItr = localArray.getIndexIterator();
113 |
114 | while( valItr.hasNext() ) {
115 | tempInt = valItr.getIntNext();
116 | avgResult.addValue(tempInt);
117 | }
118 |
119 | Utils.adjustGIDForLogicalOffset(localGID, key.getLogicalStartOffset(), extractionShape );
120 |
121 | myLongW.set( localGID.flatten(variableShape) );
122 |
123 | context.write(myLongW, avgResult);
124 |
125 | LOG.info("ArraySpec corner: " + Utils.arrayToString(key.getCorner()) +
126 | " shape: " + Utils.arrayToString(key.getShape()) +
127 | " logical start: " + Utils.arrayToString(key.getLogicalStartOffset()) +
128 | " extraction shape: " + Utils.arrayToString(extractionShape) +
129 | " localGID: " + Utils.arrayToString(localGID.getGroupID())
130 | );
131 |
132 | }
133 |
134 | timerA = System.currentTimeMillis() - timerA;
135 | LOG.info("for corner " + Utils.arrayToString(key.getCorner()) +
136 | " map loop time: " +
137 | timerA + " ms with " );
138 | } catch ( Exception e ) {
139 | System.out.println( " Exception caught in Average.map(). " + e.toString() );
140 | }
141 | }
142 | }
143 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/map/IdentityMapper.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.map;
2 |
3 | import java.io.IOException;
4 |
5 | import java.lang.Integer;
6 | import java.lang.Long;
7 | import java.util.HashMap;
8 | import java.util.Iterator;
9 | import java.util.Map;
10 |
11 | import org.apache.commons.logging.Log;
12 | import org.apache.commons.logging.LogFactory;
13 | import org.apache.hadoop.io.IntWritable;
14 | import org.apache.hadoop.io.LongWritable;
15 | import org.apache.hadoop.mapreduce.Mapper;
16 |
17 | import ucar.ma2.Array;
18 | import ucar.ma2.ArrayInt;
19 | import ucar.ma2.IndexIterator;
20 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
21 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
22 | import edu.ucsc.srl.damasc.netcdf.Utils;
23 |
24 | /**
25 | * Mapper for the Identity function
26 | * Currently uses an in-memory combiner
27 | */
28 | public class IdentityMapper extends Mapper {
29 |
30 | private static final Log LOG = LogFactory.getLog(IdentityMapper.class);
31 | private static boolean _benchmarkArraySpec = true;
32 |
33 | public static enum InvalidCell { INVALID_CELL_COUNT } ;
34 |
35 | /**
36 | * Reduces values for a given key
37 | * @param key ArraySpec representing the given Array being passed in
38 | * @param value an Array to process that corresponds to the given key
39 | * @param context the Context object for the currently executing job
40 | */
41 | public void map(ArraySpec key, Array value, Context context)
42 | throws IOException, InterruptedException {
43 |
44 | long timerA = System.currentTimeMillis();
45 |
46 | ArrayInt ncArray = (ArrayInt)value;
47 | int numCells = (int)ncArray.getSize();
48 |
49 | // this is a fair bit of memeory. Better to do it in one allocation, than a lot of them,
50 | // but we should still try to optimize this out
51 |
52 | int[] globalCoord = new int[key.getShape().length];
53 | int[] groupIDArray = new int[key.getShape().length];
54 |
55 | int[] allOnes = new int[key.getShape().length];
56 | for( int i=0; i inMapperCombiner = new HashMap();
63 |
64 | long[] longArray = new long[numCells];
65 | int[] intArray = new int[numCells];
66 |
67 | IndexIterator iter = ncArray.getIndexIterator();
68 |
69 | GroupID myGroupID = new GroupID();
70 | myGroupID.setName(key.getVarName());
71 |
72 | IntWritable myIntW = new IntWritable();
73 | LongWritable myLongW = new LongWritable();
74 |
75 | //Integer myInt = new Integer();
76 | //Long myLong = new Long();
77 |
78 | int[] extractionShape =
79 | Utils.getExtractionShape(context.getConfiguration(),
80 | key.getShape().length);
81 | int[] variableShape =
82 | Utils.getVariableShape(context.getConfiguration());
83 |
84 | int invalidMapperCounter = 0;
85 | int val = 0;
86 | int counter = 0;
87 | long startup = System.currentTimeMillis() - timerA;
88 | LOG.info("Startup time: " + startup + " ms");
89 |
90 | timerA = System.currentTimeMillis();
91 |
92 | while( iter.hasNext() ) {
93 |
94 | val = iter.getIntNext();
95 |
96 | // orient this cell's location in the global space
97 | globalCoord = Utils.mapToGlobal(iter.getCurrentCounter(),
98 | key.getCorner(), globalCoord );
99 |
100 | // track cells that are invalid. This is typically due
101 | // to filtering of cells by the query
102 | // it's only applied at the mapper for optB.
103 | // optC applies it at split generation
104 |
105 | if ( !Utils.noScanEnabled(context.getConfiguration()) )
106 | {
107 | if ( !Utils.isValid(globalCoord, context.getConfiguration()) ) {
108 | invalidMapperCounter++;
109 | continue;
110 | }
111 | }
112 |
113 | // figure out the "bucket" for this (determined by the
114 | // extraction shape and position of this cell
115 | myGroupID = Utils.mapToLocal(globalCoord, groupIDArray,
116 | myGroupID, extractionShape);
117 |
118 | longArray[counter] = myGroupID.flatten(variableShape);
119 | intArray[counter] = val;
120 |
121 | if ( inMapperCombiner.containsKey(longArray[counter]) ){
122 | if ( intArray[counter] > inMapperCombiner.get(longArray[counter]) ){
123 | inMapperCombiner.put(longArray[counter], intArray[counter]);
124 | }
125 | } else {
126 | inMapperCombiner.put(longArray[counter], intArray[counter]);
127 | }
128 |
129 | counter++;
130 | }
131 |
132 | timerA = System.currentTimeMillis() - timerA;
133 | LOG.info("for corner " + Utils.arrayToString(key.getCorner()) +
134 | " map loop time: " +
135 | timerA + " ms with " + invalidMapperCounter +
136 | " invalid ArraySpecs" );
137 |
138 | // stop the timer
139 | timerA = System.currentTimeMillis();
140 |
141 | // write out all results of the in-mapper combiner here
142 | Iterator> itr = inMapperCombiner.entrySet().iterator();
143 | while(itr.hasNext()){
144 | Map.Entry pairs = itr.next();
145 | myLongW.set(pairs.getKey());
146 | myIntW.set(pairs.getValue() );
147 | context.write(myLongW, myIntW);
148 | }
149 |
150 | context.getCounter(InvalidCell.INVALID_CELL_COUNT).increment(invalidMapperCounter);
151 | timerA = System.currentTimeMillis() - timerA;
152 | LOG.info("writing data and increment counte took " +
153 | timerA + " ms" );
154 | }
155 | }
156 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/map/MaxMapper.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.map;
2 |
3 | import java.io.IOException;
4 |
5 | import java.lang.Integer;
6 | import java.lang.Long;
7 | import java.util.HashMap;
8 | import java.util.Iterator;
9 | import java.util.Map;
10 |
11 | import org.apache.commons.logging.Log;
12 | import org.apache.commons.logging.LogFactory;
13 | import org.apache.hadoop.io.IntWritable;
14 | import org.apache.hadoop.io.LongWritable;
15 | import org.apache.hadoop.mapreduce.Mapper;
16 |
17 | import ucar.ma2.Array;
18 | import ucar.ma2.ArrayInt;
19 | import ucar.ma2.IndexIterator;
20 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
21 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
22 | import edu.ucsc.srl.damasc.netcdf.io.GroupIDGen;
23 | import edu.ucsc.srl.damasc.netcdf.Utils;
24 |
25 | /**
26 | * A Mapper class that applies the Max function
27 | */
28 | public class MaxMapper extends Mapper {
29 |
30 | private static final Log LOG = LogFactory.getLog(IdentityMapper.class);
31 | private static boolean _benchmarkArraySpec = true;
32 |
33 | public static enum InvalidCell { INVALID_CELL_COUNT } ;
34 |
35 | /**
36 | * Reduces values for a given key
37 | * @param key ArraySpec representing the given Array being passed in
38 | * @param value an Array to process that corresponds to the given key
39 | * @param context the Context object for the currently executing job
40 | */
41 | public void map(ArraySpec key, Array value, Context context)
42 | throws IOException, InterruptedException {
43 |
44 | long timerA = System.currentTimeMillis();
45 |
46 | ArrayInt ncArray = (ArrayInt)value;
47 |
48 | int[] allOnes = new int[key.getShape().length];
49 | for( int i=0; i groupSubArrayMap = new HashMap();
61 |
62 | GroupIDGen myGIDG = new GroupIDGen();
63 | GroupIDGen.pullOutSubArrays( myGIDG, ncArray, key, extractionShape,
64 | allOnes, groupSubArrayMap);
65 |
66 | ArrayInt localArray;
67 | GroupID localGID;
68 | // now roll through all the entries in the HashMap
69 |
70 | int invalidMapperCounter = 0;
71 | int currentMax = Integer.MIN_VALUE;
72 | int tempValue = Integer.MIN_VALUE;
73 |
74 | IntWritable myIntW = new IntWritable();
75 | LongWritable myLongW = new LongWritable();
76 |
77 | Iterator> gidItr =
78 | groupSubArrayMap.entrySet().iterator();
79 |
80 | while (gidItr.hasNext() ) {
81 | currentMax = Integer.MIN_VALUE;
82 | Map.Entry pairs = gidItr.next();
83 | localGID = pairs.getKey();
84 | localArray = (ArrayInt)pairs.getValue();
85 |
86 | // TODO sort out how to do filtering with this new GroupID based setup
87 | // -jbuck
88 |
89 | IndexIterator valItr = localArray.getIndexIterator();
90 |
91 | while( valItr.hasNext() ) {
92 |
93 | tempValue = valItr.getIntNext();
94 | if ( tempValue > currentMax ) {
95 | currentMax = tempValue;
96 | }
97 | }
98 |
99 | // write out the current groupID and the max value found for it
100 | //Utils.flatten(localGID);
101 | myIntW.set(currentMax);
102 | context.write(localGID, myIntW);
103 |
104 | }
105 |
106 | timerA = System.currentTimeMillis() - timerA;
107 | LOG.info("for corner " + Utils.arrayToString(key.getCorner()) +
108 | " map loop time: " +
109 | timerA + " ms with " );
110 | }
111 | }
112 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/map/MedianMapper.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.map;
2 |
3 | import java.io.IOException;
4 | import java.io.BufferedWriter;
5 | import java.io.File;
6 | import java.io.FileWriter;
7 | import java.util.HashMap;
8 | import java.util.Iterator;
9 | import java.util.Map;
10 | import java.util.Set;
11 |
12 | import org.apache.commons.logging.Log;
13 | import org.apache.commons.logging.LogFactory;
14 | import org.apache.hadoop.io.Text;
15 | import org.apache.hadoop.io.IntWritable;
16 | import org.apache.hadoop.io.LongWritable;
17 | import org.apache.hadoop.mapreduce.Mapper;
18 |
19 | import ucar.ma2.Array;
20 | import ucar.ma2.ArrayInt;
21 | import ucar.ma2.Index;
22 | import ucar.ma2.IndexIterator;
23 |
24 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
25 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
26 | import edu.ucsc.srl.damasc.netcdf.io.GroupIDGen;
27 | import edu.ucsc.srl.damasc.netcdf.io.HolisticResult;
28 | import edu.ucsc.srl.damasc.netcdf.io.Result;
29 | import edu.ucsc.srl.damasc.netcdf.Utils;
30 | import edu.ucsc.srl.damasc.netcdf.NetCDFUtils;
31 |
32 | /**
33 | * Mapper that prepares data for the Median operation
34 | */
35 | public class MedianMapper extends
36 | Mapper {
37 |
38 | private static final Log LOG = LogFactory.getLog(MedianMapper.class);
39 | private static boolean _benchmarkArraySpec = true;
40 |
41 | public static enum MedianMapStatus { FULL, NOTFULL }
42 |
43 | /**
44 | * Reduces values for a given key
45 | * @param key ArraySpec representing the given Array being passed in
46 | * @param value an Array to process that corresponds to the given key
47 | * @param context the Context object for the currently executing job
48 | */
49 | public void map(ArraySpec key, Array value, Context context)
50 | throws IOException, InterruptedException {
51 |
52 | long timerA = System.currentTimeMillis();
53 |
54 | ArrayInt ncArray = (ArrayInt)value;
55 |
56 | int[] allOnes = new int[key.getShape().length];
57 | for( int i=0; i groupSubArrayMap = new HashMap();
66 |
67 | GroupIDGen myGIDG = new GroupIDGen();
68 | GroupIDGen.pullOutSubArrays( myGIDG, ncArray, key, extractionShape,
69 | allOnes, groupSubArrayMap);
70 |
71 | String debugFileName = Utils.getDebugLogFileName(context.getConfiguration());
72 | if ( "" != debugFileName ) {
73 | LOG.info("Trying to log to " + debugFileName);
74 | NetCDFUtils.logGIDs( debugFileName, ncArray, key,
75 | extractionShape, groupSubArrayMap, LOG );
76 | }
77 |
78 | ArrayInt localArray;
79 | GroupID localGID;
80 |
81 | int currentMax = Integer.MIN_VALUE;
82 | int tempValue = Integer.MIN_VALUE;
83 | int invalidMapperCounter = 0;
84 | long totalCellsSeen = 0;
85 | long counter = 0;
86 | int finalValue = Integer.MIN_VALUE;
87 |
88 | // the two output classes
89 | HolisticResult outputHolisticResult = new HolisticResult();
90 |
91 | Iterator> gidItr =
92 | groupSubArrayMap.entrySet().iterator();
93 |
94 |
95 | while( gidItr.hasNext() ) {
96 | counter = 0;
97 | Map.Entry pair = gidItr.next();
98 | localGID = pair.getKey();
99 | localArray = (ArrayInt)pair.getValue();
100 |
101 | outputHolisticResult.clear();
102 |
103 | // create a holistic result big enough to hold all the data that it
104 | // may see (note, we know how many it will see but
105 | // this seems like a safe move)
106 | outputHolisticResult.setNeededValueCount(
107 | Utils.calcTotalSize(extractionShape) );
108 |
109 | IndexIterator valItr = localArray.getIndexIterator();
110 |
111 | LOG.info("exShapeSize: " + Utils.calcTotalSize(extractionShape) +
112 | " holisticResultSize: " + localArray.getSize() );
113 |
114 | while( valItr.hasNext() ) {
115 | //LOG.info("gid: " + Utils.arrayToString( localGID.getGroupID()) +
116 | // " adding element " + counter + " at: " +
117 | // outputHolisticResult.getCurrentValueCount());
118 | // get a legit value first
119 | outputHolisticResult.setValue(valItr.getIntNext());
120 | counter++;
121 | }
122 |
123 | // if we have a full HolisticResult,
124 | // we should consolidate it into a final answer
125 | // prior to writing it out
126 | if( outputHolisticResult.isFull() ){
127 | outputHolisticResult.sort();
128 |
129 | if ( !Utils.isSorted( outputHolisticResult.getValues())) {
130 | LOG.error("Holistic result for GID: " + key + " has unsorted results");
131 | }
132 |
133 | finalValue = outputHolisticResult.getValues()[outputHolisticResult.getValues().length/2 ];
134 | outputHolisticResult.setFinal(finalValue);
135 | context.getCounter(MedianMapStatus.FULL).increment(1);
136 | LOG.info("GID " + localGID + " is full in the mapper");
137 | } else {
138 | context.getCounter(MedianMapStatus.NOTFULL).increment(1);
139 | LOG.info("GID " + localGID + " is NOT full in the mapper, has " +
140 | outputHolisticResult.getCurrentValueCount() + " elements" );
141 | }
142 |
143 | context.write(localGID, outputHolisticResult);
144 |
145 | totalCellsSeen += localArray.getSize();
146 | }
147 |
148 | timerA = System.currentTimeMillis() - timerA;
149 | LOG.info("for corner " + Utils.arrayToString(key.getCorner()) +
150 | " map loop time: " +
151 | timerA + " ms with " +
152 | + totalCellsSeen + " total cells seen");
153 | }
154 | }
155 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/map/NullMapper.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.map;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 | import org.apache.hadoop.io.IntWritable;
8 | import org.apache.hadoop.io.LongWritable;
9 | import org.apache.hadoop.mapreduce.Mapper;
10 |
11 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
12 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
13 |
14 | import ucar.ma2.Array;
15 | import ucar.ma2.ArrayInt;
16 |
17 | /**
18 | * Dummy mapper, just passed data through with a dummy key.
19 | * This is used for testing purposes
20 | */
21 | public class NullMapper extends Mapper {
22 |
23 | /**
24 | * Reduces values for a given key
25 | * @param key ArraySpec representing the given Array being passed in
26 | * @param value an Array to process that corresponds to the given key
27 | * @param context the Context object for the currently executing job
28 | */
29 | public void map(ArraySpec key, Array value, Context context)
30 | throws IOException, InterruptedException {
31 | try {
32 | ArrayInt intArray = (ArrayInt)value;
33 | int[] dummyGID = {0};
34 | GroupID groupID = new GroupID(dummyGID, "nullData");
35 | IntWritable intW = new IntWritable(Integer.MIN_VALUE);
36 |
37 | context.write(groupID, intW);
38 | } catch ( Exception e ) {
39 | System.out.println("Caught an exception in NullMapper.map()" + e.toString() );
40 | }
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/map/SimpleMaxMapper.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.map;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 | import org.apache.hadoop.io.Text;
8 | import org.apache.hadoop.io.IntWritable;
9 | import org.apache.hadoop.io.LongWritable;
10 | import org.apache.hadoop.mapreduce.Mapper;
11 |
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
14 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
15 | import edu.ucsc.srl.damasc.netcdf.io.Result;
16 | import edu.ucsc.srl.damasc.netcdf.Utils;
17 |
18 | import ucar.ma2.Array;
19 | import ucar.ma2.ArrayInt;
20 | import ucar.ma2.Index;
21 | import ucar.ma2.IndexIterator;
22 |
23 | /**
24 | * Mapper for the Max function that uses simple data structures as
25 | * keys
26 | */
27 | public class SimpleMaxMapper
28 | extends Mapper {
29 |
30 | private static final Log LOG = LogFactory.getLog(SimpleMaxMapper.class);
31 | private static boolean _benchmarkArraySpec = true;
32 |
33 | public static enum InvalidCell { INVALID_CELL_COUNT } ;
34 |
35 | /**
36 | * Reduces values for a given key
37 | * @param key ArraySpec representing the given Array being passed in
38 | * @param value an Array to process that corresponds to the given key
39 | * @param context the Context object for the currently executing job
40 | */
41 | public void map(ArraySpec key, Array value, Context context)
42 | throws IOException, InterruptedException {
43 |
44 | ArrayInt intArray = (ArrayInt)value;
45 |
46 | int[] globalCoord = new int[key.getShape().length];
47 | int[] groupIDArray = new int[key.getShape().length];
48 |
49 | int[] allOnes = new int[key.getShape().length];
50 | for( int i=0; i {
28 |
29 | private static final Log LOG = LogFactory.getLog(SimpleMedianMapper.class);
30 | private static boolean _benchmarkArraySpec = true;
31 |
32 | static enum MapOutputsCreated{ MAP }
33 |
34 | public static enum InvalidCell { INVALID_CELL_COUNT } ;
35 |
36 |
37 | /**
38 | * Reduces values for a given key
39 | * @param key ArraySpec representing the given Array being passed in
40 | * @param value an Array to process that corresponds to the given key
41 | * @param context the Context object for the currently executing job
42 | */
43 | public void map(ArraySpec key, Array value, Context context)
44 | throws IOException, InterruptedException {
45 |
46 | ArrayInt intArray = (ArrayInt)value;
47 |
48 | int[] globalCoord = new int[key.getShape().length];
49 | int[] groupIDArray = new int[key.getShape().length];
50 |
51 | int[] allOnes = new int[key.getShape().length];
52 | for( int i=0; i {
21 |
22 | private static final Log LOG = LogFactory.getLog(AverageReducer.class);
23 |
24 | /**
25 | * Reduces all the values for the given key, produces the average of the
26 | * AverageResult objects in values
27 | * @param key the flattened corner for this instance of the extraction shape
28 | * in the global logical space
29 | * @param values an Iterable list of AverageResult objects that represent all the inputs
30 | * for this key
31 | * @param context the Context object for the executing program
32 | */
33 | public void reduce(LongWritable key, Iterable values,
34 | Context context)
35 | throws IOException, InterruptedException {
36 |
37 | long timer = System.currentTimeMillis();
38 |
39 | int[] variableShape = Utils.getVariableShape( context.getConfiguration());
40 | AverageResult avgResult = new AverageResult();
41 | GroupID myGroupID = new GroupID();
42 | IntWritable myIntW = new IntWritable();
43 |
44 | //for (Result value : values) {
45 | for (AverageResult value : values) {
46 | //currentAverage = (int) ((((long)currentAverage * currentSamplesInAverage) + tempValue) / (currentSamplesInAverage++));
47 | avgResult.addAverageResult(value);
48 | }
49 |
50 | myGroupID.setGroupID( myGroupID.unflatten(variableShape, key.get()) );
51 | myIntW.set(avgResult.getCurrentValue());
52 | context.write(myGroupID, myIntW);
53 |
54 | timer = System.currentTimeMillis() - timer;
55 | LOG.info("total reducer took: " + timer + " ms");
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/reduce/IdentityReducer.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.reduce;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.IntWritable;
9 | import org.apache.hadoop.io.LongWritable;
10 | import org.apache.hadoop.mapreduce.Reducer;
11 |
12 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
13 | import edu.ucsc.srl.damasc.netcdf.Utils;
14 |
15 | public class IdentityReducer extends
16 | Reducer {
17 |
18 | private static final Log LOG = LogFactory.getLog(IdentityReducer.class);
19 |
20 | /**
21 | * Reduces all the values for the given key, produces the Identity of the
22 | * IntWritable objects in values
23 | * @param key the flattened corner for this instance of the extraction shape
24 | * in the global logical space
25 | * @param values an Iterable list of IntWritable objects that represent all the inputs
26 | * for this key
27 | * @param context the Context object for the executing program
28 | */
29 | public void reduce(LongWritable key, Iterable values,
30 | Context context)
31 | throws IOException, InterruptedException {
32 |
33 | long timer = System.currentTimeMillis();
34 | GroupID tempID = new GroupID();
35 |
36 | // now we need to parse the variable dimensions out
37 | int[] variableShape = Utils.getVariableShape( context.getConfiguration());
38 |
39 | IntWritable maxVal = new IntWritable();
40 | maxVal.set(Integer.MIN_VALUE);
41 |
42 | for (IntWritable value : values) {
43 | if ( value.get() > maxVal.get() )
44 | maxVal.set(value.get());
45 | }
46 |
47 | tempID.setGroupID( tempID.unflatten(variableShape, key.get() ) );
48 | context.write(tempID, maxVal);
49 |
50 | timer = System.currentTimeMillis() - timer;
51 | LOG.info("total reducer took: " + timer + " ms");
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/reduce/MaxReducer.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.reduce;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.IntWritable;
9 | import org.apache.hadoop.io.LongWritable;
10 | import org.apache.hadoop.mapreduce.Reducer;
11 |
12 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
13 | import edu.ucsc.srl.damasc.netcdf.Utils;
14 |
15 | /**
16 | * Reducer for the Max operator
17 | */
18 | public class MaxReducer extends
19 | Reducer {
20 |
21 | private static final Log LOG = LogFactory.getLog(IdentityReducer.class);
22 |
23 | /**
24 | * Reduces all the values for the given key, produces the maximum of the
25 | * IntWritable objects in values
26 | * @param key the flattened corner for this instance of the extraction shape
27 | * in the global logical space
28 | * @param values an Iterable list of IntWritable objects that represent all the inputs
29 | * for this key
30 | * @param context the Context object for the executing program
31 | */
32 | public void reduce(GroupID key, Iterable values,
33 | Context context)
34 | throws IOException, InterruptedException {
35 |
36 | long timer = System.currentTimeMillis();
37 |
38 | // now we need to parse the variable dimensions out
39 | //int[] variableShape = Utils.getVariableShape( context.getConfiguration());
40 |
41 | IntWritable maxVal = new IntWritable();
42 | maxVal.set(Integer.MIN_VALUE);
43 |
44 | for (IntWritable value : values) {
45 | if ( value.get() > maxVal.get() )
46 | maxVal.set(value.get());
47 | }
48 |
49 | //tempID.setGroupID( tempID.unflatten(variableShape, key.get() ) );
50 | context.write(key, maxVal);
51 |
52 | timer = System.currentTimeMillis() - timer;
53 | LOG.info("total reducer took: " + timer + " ms");
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/reduce/MedianReducer.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.reduce;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.Text;
9 | import org.apache.hadoop.io.IntWritable;
10 | import org.apache.hadoop.io.LongWritable;
11 | import org.apache.hadoop.mapreduce.Reducer;
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
14 | import edu.ucsc.srl.damasc.netcdf.io.HolisticResult;
15 | import edu.ucsc.srl.damasc.netcdf.io.Result;
16 |
17 | import edu.ucsc.srl.damasc.netcdf.Utils;
18 |
19 | /**
20 | * Reducer for the Median operator
21 | */
22 | public class MedianReducer extends
23 | Reducer {
24 |
25 | private static final Log LOG = LogFactory.getLog(MedianReducer.class);
26 |
27 | static enum MedianReducerStatus { FULL, NOTFULL, NOTFINAL }
28 |
29 | /**
30 | * Reduces all the values for the given key, produces the median of the
31 | * HolisticResult objects in values
32 | * @param key the flattened corner for this instance of the extraction shape
33 | * in the global logical space
34 | * @param values an Iterable list of HolisticResult objects that represent all the inputs
35 | * for this key
36 | * @param context the Context object for the executing program
37 | */
38 | public void reduce(GroupID key, Iterable values,
39 | Context context)
40 | throws IOException, InterruptedException {
41 |
42 | //GroupID tempID = new GroupID();
43 | IntWritable outputInt = new IntWritable(Integer.MIN_VALUE);
44 |
45 | // now we need to parse the variable dimensions out
46 | int[] variableShape = Utils.getVariableShape( context.getConfiguration());
47 | int[] extractionShape =
48 | Utils.getExtractionShape(context.getConfiguration(),
49 | variableShape.length);
50 |
51 | int neededSize = Utils.calcTotalSize( extractionShape );
52 |
53 | HolisticResult maxVal = new HolisticResult();
54 | maxVal.setNeededValueCount( neededSize );
55 |
56 | for (HolisticResult value : values) {
57 | // sanity check
58 | if ( maxVal.isFull() ) {
59 | LOG.warn("Adding an element to an already full HR. Key: " +
60 | key.toString() +
61 | " array size: " + maxVal.getNeededValueCount() +
62 | " current elems: " +
63 | maxVal.getCurrentValueCount() );
64 | }
65 |
66 | LOG.info("GID: " + key + " merging in " + value.getCurrentValueCount() +
67 | " keys, already " + maxVal.getCurrentValueCount() + " present");
68 |
69 | maxVal.merge(value);
70 | }
71 |
72 | // now, the remainig holistic result should be full. Check though
73 | // and make sure it wasn't already finalized
74 |
75 | if( maxVal.isFull() && !maxVal.isFinal() ) {
76 | // apply whatever function you want,
77 | // in this case we sort and then pull the median out
78 | maxVal.sort();
79 |
80 | if ( !Utils.isSorted( maxVal.getValues() )) {
81 | LOG.error("Holistic result for GID: " + key + " has unsorted results");
82 | }
83 |
84 | maxVal.setFinal( maxVal.getValues()[(maxVal.getValues().length)/2] );
85 | //LOG.info("gid: " + key + " is full at " +
86 | // maxVal.getCurrentValueCount() + " elements");
87 | context.getCounter(MedianReducerStatus.FULL).increment(1);
88 | } else if (!maxVal.isFull() ) {
89 | LOG.info("gid: " + key + " has " + maxVal.getCurrentValueCount() +
90 | " elements" +
91 | " but should be full");
92 | context.getCounter(MedianReducerStatus.NOTFULL).increment(1);
93 | } else if (maxVal.isFinal() ) {
94 | LOG.info("gid: " + key + " has already been set to final");
95 | context.getCounter(MedianReducerStatus.NOTFINAL).increment(1);
96 | }
97 |
98 |
99 | //tempID.setGroupID( tempID.unflatten(variableShape, key.get() ) );
100 |
101 | outputInt.set(maxVal.getValue(0));
102 | context.write(key, outputInt);
103 | }
104 | }
105 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/reduce/NullReducer.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.reduce;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.IntWritable;
9 | import org.apache.hadoop.io.LongWritable;
10 | import org.apache.hadoop.mapreduce.Reducer;
11 |
12 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
13 | import edu.ucsc.srl.damasc.netcdf.Utils;
14 |
15 | /**
16 | * Reducer that simply iterates through the data it is passed
17 | */
18 | public class NullReducer extends
19 | Reducer {
20 |
21 | private static final Log LOG = LogFactory.getLog(NullReducer.class);
22 |
23 | /**
24 | * Iterates through the data it is passed, doing nothing to it. Outputs a
25 | * Integer.MINIMUM_VALUE as the value for its key
26 | * @param key the flattened corner for this instance of the extraction shape
27 | * in the global logical space
28 | * @param values an Iterable list of IntWritable objects that represent all the inputs
29 | * for this key
30 | * @param context the Context object for the executing program
31 | */
32 | public void reduce(GroupID key, Iterable values,
33 | Context context)
34 | throws IOException, InterruptedException {
35 |
36 | long timer = System.currentTimeMillis();
37 |
38 | IntWritable maxVal = new IntWritable();
39 | maxVal.set(Integer.MIN_VALUE);
40 |
41 | // empty loop
42 | for (IntWritable value : values) {
43 | }
44 |
45 | context.write(key, maxVal);
46 |
47 | timer = System.currentTimeMillis() - timer;
48 | LOG.info("total reducer took: " + timer + " ms");
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/reduce/SimpleMaxReducer.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.reduce;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.Text;
9 | import org.apache.hadoop.io.IntWritable;
10 | import org.apache.hadoop.io.LongWritable;
11 | import org.apache.hadoop.mapreduce.Reducer;
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
14 | import edu.ucsc.srl.damasc.netcdf.io.Result;
15 |
16 | import edu.ucsc.srl.damasc.netcdf.Utils;
17 |
18 | /**
19 | * A reducer that applies the maximum function. Uses a simple data structure
20 | * as its key
21 | */
22 | public class SimpleMaxReducer extends
23 | Reducer {
24 |
25 | private static final Log LOG = LogFactory.getLog(SimpleMaxReducer.class);
26 |
27 | /**
28 | * Reduces all the values for the given key, produces the average of the
29 | * IntWritable objects in values
30 | * @param key the flattened corner for this instance of the extraction shape
31 | * in the global logical space
32 | * @param values an Iterable list of IntWritable objects that represent all the inputs
33 | * for this key
34 | * @param context the Context object for the executing program
35 | */
36 | public void reduce(LongWritable key, Iterable values,
37 | Context context)
38 | throws IOException, InterruptedException {
39 |
40 | GroupID tempID = new GroupID();
41 |
42 | // now we need to parse the variable dimensions out
43 | int[] variableShape = Utils.getVariableShape( context.getConfiguration());
44 |
45 | IntWritable maxVal = new IntWritable();
46 | maxVal.set(Integer.MIN_VALUE);
47 |
48 | for (IntWritable value : values) {
49 | if ( value.get() > maxVal.get() )
50 | maxVal.set(value.get());
51 | }
52 |
53 | tempID.setGroupID( tempID.unflatten(variableShape, key.get() ) );
54 | context.write(tempID, maxVal);
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/reduce/SimpleMedianReducer.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.reduce;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 |
8 | import org.apache.hadoop.io.Text;
9 | import org.apache.hadoop.io.IntWritable;
10 | import org.apache.hadoop.io.LongWritable;
11 | import org.apache.hadoop.mapreduce.Reducer;
12 |
13 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
14 | import edu.ucsc.srl.damasc.netcdf.io.HolisticResult;
15 | import edu.ucsc.srl.damasc.netcdf.io.Result;
16 |
17 | import edu.ucsc.srl.damasc.netcdf.Utils;
18 |
19 | /**
20 | * Reducer that applies the median function. Uses simple data structures as keys
21 | */
22 | public class SimpleMedianReducer extends
23 | Reducer {
24 |
25 | private static final Log LOG = LogFactory.getLog(SimpleMedianReducer.class);
26 |
27 | static enum SimpleMedianReducerStatus { FULL, NOTFULL }
28 |
29 | /**
30 | * Reduces all the values for the given key, produces the median of the
31 | * HolisticResult objects in values
32 | * @param key the flattened corner for this instance of the extraction shape
33 | * in the global logical space
34 | * @param values an Iterable list of HolisticResult objects that represent all the inputs
35 | * for this key
36 | * @param context the Context object for the executing program
37 | */
38 | public void reduce(LongWritable key, Iterable values,
39 | Context context)
40 | throws IOException, InterruptedException {
41 |
42 | GroupID tempID = new GroupID();
43 | IntWritable outputInt = new IntWritable(Integer.MIN_VALUE);
44 |
45 | // now we need to parse the variable dimensions out
46 | int[] variableShape = Utils.getVariableShape( context.getConfiguration());
47 | int[] extractionShape =
48 | Utils.getExtractionShape(context.getConfiguration(),
49 | variableShape.length);
50 |
51 | int neededSize = Utils.calcTotalSize( extractionShape );
52 |
53 | HolisticResult maxVal = new HolisticResult();
54 | maxVal.setNeededValueCount( neededSize );
55 |
56 | for (HolisticResult value : values) {
57 | // sanity check
58 | if ( maxVal.isFull() ) {
59 | LOG.warn("Adding an element to an already full HR. Key: " +
60 | key.toString() +
61 | " array size: " + maxVal.getNeededValueCount() +
62 | " current elems: " +
63 | maxVal.getCurrentValueCount() );
64 | }
65 |
66 | maxVal.merge(value);
67 | }
68 |
69 | // now, the remainig holistic result should be full. Check though
70 | // and make sure it wasn't already finalized
71 |
72 | if( maxVal.isFull() && !maxVal.isFinal() ) {
73 | // apply whatever function you want,
74 | // in this case we sort and then pull the median out
75 | maxVal.sort();
76 | maxVal.setFinal( maxVal.getValues()[(maxVal.getValues().length)/2] );
77 | LOG.info("gid: " + key + " is full at " +
78 | maxVal.getCurrentValueCount() + " elements");
79 | context.getCounter(SimpleMedianReducerStatus.FULL).increment(1);
80 | } else if (!maxVal.isFull() ) {
81 | LOG.info("gid: " + key + " has " + maxVal.getCurrentValueCount() +
82 | " elements" +
83 | " but should be full");
84 | } else if (maxVal.isFinal() ) {
85 | LOG.info("gid: " + key + " has already been set to final");
86 | }
87 |
88 |
89 | tempID.setGroupID( tempID.unflatten(variableShape, key.get() ) );
90 |
91 | outputInt.set(maxVal.getValue(0));
92 | context.write(tempID, outputInt);
93 | }
94 | }
95 |
--------------------------------------------------------------------------------
/src/edu/ucsc/srl/damasc/netcdf/tools/Average.java:
--------------------------------------------------------------------------------
1 | package edu.ucsc.srl.damasc.netcdf.tools;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.hadoop.conf.Configuration;
6 | import org.apache.hadoop.conf.Configured;
7 | import org.apache.hadoop.fs.Path;
8 | import org.apache.hadoop.io.Text;
9 | import org.apache.hadoop.io.IntWritable;
10 | import org.apache.hadoop.io.LongWritable;
11 | //import org.apache.hadoop.mapred.JobConf;
12 | import org.apache.hadoop.mapreduce.*;
13 | import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
14 | import org.apache.hadoop.util.Tool;
15 | import org.apache.hadoop.util.ToolRunner;
16 |
17 | import edu.ucsc.srl.damasc.netcdf.combine.AverageCombiner;
18 | import edu.ucsc.srl.damasc.netcdf.io.input.NetCDFFileInputFormat;
19 | import edu.ucsc.srl.damasc.netcdf.io.GroupID;
20 | import edu.ucsc.srl.damasc.netcdf.io.AverageResult;
21 | import edu.ucsc.srl.damasc.netcdf.io.ArraySpec;
22 | import edu.ucsc.srl.damasc.netcdf.map.AverageMapper;
23 | import edu.ucsc.srl.damasc.netcdf.reduce.AverageReducer;
24 | import edu.ucsc.srl.damasc.netcdf.Utils;
25 | import edu.ucsc.srl.damasc.netcdf.Utils.Operator;
26 |
27 | public class Average extends Configured implements Tool {
28 |
29 | public int run(String[] args) throws Exception {
30 | if (args.length != 2) {
31 | System.err.println("Usage: identity