1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 
19 package org.apache.hadoop.streaming;
20 
21 import java.io.File;
22 import java.io.IOException;
23 import java.io.DataOutputStream;
24 import java.util.Map;
25 import java.util.zip.ZipEntry;
26 import java.util.zip.ZipOutputStream;
27 
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.fs.FileSystem;
33 import org.apache.hadoop.fs.FileUtil;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.mapred.*;
36 import org.apache.hadoop.hdfs.MiniDFSCluster;
37 
38 /**
39  * This class tests cacheArchive option of streaming
40  * The test case creates 2 archive files, ships it with hadoop
41  * streaming and compares the output with expected output
42  */
43 public class TestMultipleArchiveFiles extends TestStreaming
44 {
45   private static final Log LOG = LogFactory.getLog(TestMultipleArchiveFiles.class);
46 
47   private StreamJob job;
48   private String INPUT_DIR = "multiple-archive-files/";
49   private String INPUT_FILE = INPUT_DIR + "input.txt";
50   private String CACHE_ARCHIVE_1 = INPUT_DIR + "cacheArchive1.zip";
51   private File CACHE_FILE_1 = null;
52   private String CACHE_ARCHIVE_2 = INPUT_DIR + "cacheArchive2.zip";
53   private File CACHE_FILE_2 = null;
54   private String expectedOutput = null;
55   private String OUTPUT_DIR = "out";
56   private Configuration conf = null;
57   private MiniDFSCluster dfs = null;
58   private MiniMRCluster mr = null;
59   private FileSystem fileSys = null;
60   private String namenode = null;
61 
TestMultipleArchiveFiles()62   public TestMultipleArchiveFiles() throws Exception {
63     CACHE_FILE_1 = new File("cacheArchive1");
64     CACHE_FILE_2 = new File("cacheArchive2");
65     input = "HADOOP";
66     expectedOutput = "HADOOP\t\nHADOOP\t\n";
67     conf = new Configuration();
68     dfs = new MiniDFSCluster.Builder(conf).build();
69     fileSys = dfs.getFileSystem();
70     namenode = fileSys.getUri().getAuthority();
71     mr  = new MiniMRCluster(1, namenode, 1);
72 
73     map = XARGS_CAT;
74     reduce = CAT;
75   }
76 
77   @Override
setInputOutput()78   protected void setInputOutput() {
79     inputFile = INPUT_FILE;
80     outDir = OUTPUT_DIR;
81   }
82 
createInput()83   protected void createInput() throws IOException
84   {
85     fileSys.delete(new Path(INPUT_DIR), true);
86     DataOutputStream dos = fileSys.create(new Path(INPUT_FILE));
87     String inputFileString = "symlink1" + File.separator
88       + "cacheArchive1\nsymlink2" + File.separator + "cacheArchive2";
89     dos.write(inputFileString.getBytes("UTF-8"));
90     dos.close();
91 
92     DataOutputStream out = fileSys.create(new Path(CACHE_ARCHIVE_1.toString()));
93     ZipOutputStream zos = new ZipOutputStream(out);
94     ZipEntry ze = new ZipEntry(CACHE_FILE_1.toString());
95     zos.putNextEntry(ze);
96     zos.write(input.getBytes("UTF-8"));
97     zos.closeEntry();
98     zos.close();
99 
100     out = fileSys.create(new Path(CACHE_ARCHIVE_2.toString()));
101     zos = new ZipOutputStream(out);
102     ze = new ZipEntry(CACHE_FILE_2.toString());
103     zos.putNextEntry(ze);
104     zos.write(input.getBytes("UTF-8"));
105     zos.closeEntry();
106     zos.close();
107   }
108 
genArgs()109   protected String[] genArgs() {
110     String workDir = fileSys.getWorkingDirectory().toString() + "/";
111     String cache1 = workDir + CACHE_ARCHIVE_1 + "#symlink1";
112     String cache2 = workDir + CACHE_ARCHIVE_2 + "#symlink2";
113 
114     for (Map.Entry<String, String> entry : mr.createJobConf()) {
115       args.add("-jobconf");
116       args.add(entry.getKey() + "=" + entry.getValue());
117     }
118     args.add("-jobconf");
119     args.add("mapreduce.job.reduces=1");
120     args.add("-cacheArchive");
121     args.add(cache1);
122     args.add("-cacheArchive");
123     args.add(cache2);
124     args.add("-jobconf");
125     args.add("mapred.jar=" + STREAMING_JAR);
126     return super.genArgs();
127   }
128 
checkOutput()129   protected void checkOutput() throws IOException {
130     StringBuffer output = new StringBuffer(256);
131     Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
132                                             new Path(OUTPUT_DIR)));
133     for (int i = 0; i < fileList.length; i++){
134       LOG.info("Adding output from file: " + fileList[i]);
135       output.append(StreamUtil.slurpHadoop(fileList[i], fileSys));
136     }
137     assertOutput(expectedOutput, output.toString());
138   }
139 }
140