001    /**
002     * Copyright (c) 2010 Yahoo! Inc. All rights reserved.
003     * Licensed under the Apache License, Version 2.0 (the "License");
004     * you may not use this file except in compliance with the License.
005     * You may obtain a copy of the License at
006     *
007     *   http://www.apache.org/licenses/LICENSE-2.0
008     *
009     *  Unless required by applicable law or agreed to in writing, software
010     *  distributed under the License is distributed on an "AS IS" BASIS,
011     *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
012     *  See the License for the specific language governing permissions and
013     *  limitations under the License. See accompanying LICENSE file.
014     */
015    package org.apache.oozie.service;
016    
017    import org.apache.hadoop.conf.Configuration;
018    import org.apache.hadoop.fs.FileStatus;
019    import org.apache.hadoop.fs.FileSystem;
020    import org.apache.hadoop.fs.Path;
021    import org.apache.hadoop.fs.PathFilter;
022    import org.apache.oozie.client.OozieClient;
023    import org.apache.oozie.client.XOozieClient;
024    import org.apache.oozie.command.CommandException;
025    import org.apache.oozie.workflow.WorkflowApp;
026    import org.apache.oozie.workflow.WorkflowException;
027    import org.apache.oozie.util.IOUtils;
028    import org.apache.oozie.util.XConfiguration;
029    import org.apache.oozie.util.XLog;
030    import org.apache.oozie.ErrorCode;
031    
032    import java.io.IOException;
033    import java.io.InputStreamReader;
034    import java.io.Reader;
035    import java.io.StringWriter;
036    import java.net.URI;
037    import java.net.URISyntaxException;
038    import java.util.ArrayList;
039    import java.util.List;
040    import java.util.Map;
041    
042    /**
043     * Service that provides application workflow definition reading from the path and creation of the proto configuration.
044     */
045    public abstract class WorkflowAppService implements Service {
046        public static final String APP_LIB_PATH_LIST = "oozie.wf.application.lib";
047    
048        public static final String HADOOP_UGI = "hadoop.job.ugi";
049    
050        public static final String HADOOP_USER = "user.name";
051    
052        public static final String HADOOP_JT_KERBEROS_NAME = "mapreduce.jobtracker.kerberos.principal";
053    
054        public static final String HADOOP_NN_KERBEROS_NAME = "dfs.namenode.kerberos.principal";
055    
056        /**
057         * Initialize the workflow application service.
058         *
059         * @param services services instance.
060         */
061        public void init(Services services) {
062        }
063    
064        /**
065         * Destroy the workflow application service.
066         */
067        public void destroy() {
068        }
069    
070        /**
071         * Return the public interface for workflow application service.
072         *
073         * @return {@link WorkflowAppService}.
074         */
075        public Class<? extends Service> getInterface() {
076            return WorkflowAppService.class;
077        }
078    
079        /**
080         * Read workflow definition.
081         *
082         * @param appPath application path.
083         * @param user user name.
084         * @param group group name.
085         * @param autToken authentication token.
086         * @return workflow definition.
087         * @throws WorkflowException thrown if the definition could not be read.
088         */
089        protected String readDefinition(String appPath, String user, String group, String autToken)
090                throws WorkflowException {
091            try {
092                URI uri = new URI(appPath);
093                FileSystem fs = Services.get().get(HadoopAccessorService.class).
094                        createFileSystem(user, group, uri, new Configuration());
095                Reader reader = new InputStreamReader(fs.open(new Path(uri.getPath(), "workflow.xml")));
096                StringWriter writer = new StringWriter();
097                IOUtils.copyCharStream(reader, writer);
098                return writer.toString();
099    
100            }
101            catch (IOException ex) {
102                throw new WorkflowException(ErrorCode.E0710, ex.getMessage(), ex);
103            }
104            catch (URISyntaxException ex) {
105                throw new WorkflowException(ErrorCode.E0711, appPath, ex.getMessage(), ex);
106            }
107            catch (HadoopAccessorException ex) {
108                throw new WorkflowException(ex);
109            }
110            catch (Exception ex) {
111                throw new WorkflowException(ErrorCode.E0710, ex.getMessage(), ex);
112            }
113        }
114    
115        /**
116         * Create proto configuration. <p/> The proto configuration includes the user,group and the paths which need to be
117         * added to distributed cache. These paths include .jar,.so and the resource file paths.
118         *
119         * @param jobConf job configuration.
120         * @param authToken authentication token.
121         * @return proto configuration.
122         * @throws WorkflowException thrown if the proto action configuration could not be created.
123         */
124        public XConfiguration createProtoActionConf(Configuration jobConf, String authToken) throws WorkflowException {
125            XConfiguration conf = new XConfiguration();
126            try {
127                String user = jobConf.get(OozieClient.USER_NAME);
128                String group = jobConf.get(OozieClient.GROUP_NAME);
129                String hadoopUgi = user + "," + group;
130    
131                conf.set(OozieClient.USER_NAME, user);
132                conf.set(OozieClient.GROUP_NAME, group);
133                conf.set(HADOOP_UGI, hadoopUgi);
134    
135                if (Services.get().getConf().getBoolean("oozie.service.HadoopAccessorService.kerberos.enabled", false)) {
136                    conf.set(HADOOP_JT_KERBEROS_NAME, jobConf.get(HADOOP_JT_KERBEROS_NAME));
137                    conf.set(HADOOP_NN_KERBEROS_NAME, jobConf.get(HADOOP_NN_KERBEROS_NAME));
138                }
139    
140                URI uri = new URI(jobConf.get(OozieClient.APP_PATH));
141    
142                FileSystem fs = Services.get().get(HadoopAccessorService.class).createFileSystem(user, group, uri, conf);
143    
144                Path appPath = new Path(uri.getPath());
145                XLog.getLog(getClass()).debug("jobConf.libPath = " + jobConf.get(XOozieClient.LIBPATH));
146                XLog.getLog(getClass()).debug("jobConf.appPath = " + appPath);
147    
148                List<String> filePaths = null;
149                if (jobConf.get(XOozieClient.LIBPATH) != null) { // This is a HTTP submission job;
150                    filePaths = getLibFiles(fs, appPath);
151                } else {
152                    filePaths = getLibFiles(fs, new Path(appPath + "/lib"));
153                }
154    
155                conf.setStrings(APP_LIB_PATH_LIST, filePaths.toArray(new String[filePaths.size()]));
156    
157                //Add all properties start with 'oozie.'
158                for (Map.Entry<String, String> entry : jobConf) {
159                    if (entry.getKey().startsWith("oozie.")) {
160                        String name = entry.getKey();
161                        String value = entry.getValue();
162                        conf.set(name, value);
163                    }
164                }
165                return conf;
166            }
167            catch (IOException ex) {
168                throw new WorkflowException(ErrorCode.E0712, jobConf.get(OozieClient.APP_PATH), ex.getMessage(), ex);
169            }
170            catch (URISyntaxException ex) {
171                throw new WorkflowException(ErrorCode.E0711, jobConf.get(OozieClient.APP_PATH), ex.getMessage(), ex);
172            }
173            catch (HadoopAccessorException ex) {
174                throw new WorkflowException(ex);
175            }
176            catch (Exception ex) {
177                throw new WorkflowException(ErrorCode.E0712, jobConf.get(OozieClient.APP_PATH),
178                                            ex.getMessage(), ex);
179            }
180        }
181    
182        /**
183         * Parse workflow definition.
184         *
185         * @param jobConf job configuration.
186         * @param authToken authentication token.
187         * @return workflow application.
188         * @throws WorkflowException thrown if the workflow application could not be parsed.
189         */
190        public abstract WorkflowApp parseDef(Configuration jobConf, String authToken) throws WorkflowException;
191    
192        /**
193         * Parse workflow definition.
194         * @param wfXml workflow.
195         * @return workflow application.
196         * @throws WorkflowException thrown if the workflow application could not be parsed.
197         */
198        public abstract WorkflowApp parseDef(String wfXml) throws WorkflowException;
199    
200        /**
201         * Get all library paths.
202         *
203         * @param fs file system object.
204         * @param libPath hdfs library path.
205         * @return list of paths.
206         * @throws IOException thrown if the lib paths could not be obtained.
207         */
208        private List<String> getLibFiles(FileSystem fs, Path libPath) throws IOException {
209            List<String> libPaths = new ArrayList<String>();
210            FileStatus[] files = fs.listStatus(libPath, new NoPathFilter());
211    
212            for (FileStatus file : files) {
213                libPaths.add((String) file.getPath().toUri().getPath().trim());
214            }
215            return libPaths;
216        }
217    
218        /*
219         * Filter class doing no filtering.
220         * We dont need define this class, but seems fs.listStatus() is not working properly without this.
221         * So providing this dummy no filtering Filter class.
222         */
223        private class NoPathFilter implements PathFilter {
224            @Override
225            public boolean accept(Path path) {
226                return true;
227            }
228        }
229    }