001    /**
002    * Licensed to the Apache Software Foundation (ASF) under one
003    * or more contributor license agreements.  See the NOTICE file
004    * distributed with this work for additional information
005    * regarding copyright ownership.  The ASF licenses this file
006    * to you under the Apache License, Version 2.0 (the
007    * "License"); you may not use this file except in compliance
008    * with the License.  You may obtain a copy of the License at
009    *
010    *     http://www.apache.org/licenses/LICENSE-2.0
011    *
012    * Unless required by applicable law or agreed to in writing, software
013    * distributed under the License is distributed on an "AS IS" BASIS,
014    * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015    * See the License for the specific language governing permissions and
016    * limitations under the License.
017    */
018    
019    package org.apache.hadoop.yarn.logaggregation;
020    
021    import java.io.DataInput;
022    import java.io.DataInputStream;
023    import java.io.DataOutput;
024    import java.io.DataOutputStream;
025    import java.io.EOFException;
026    import java.io.File;
027    import java.io.FileInputStream;
028    import java.io.IOException;
029    import java.io.InputStreamReader;
030    import java.io.PrintStream;
031    import java.io.Writer;
032    import java.security.PrivilegedExceptionAction;
033    import java.util.ArrayList;
034    import java.util.Arrays;
035    import java.util.Collections;
036    import java.util.EnumSet;
037    import java.util.HashMap;
038    import java.util.List;
039    import java.util.Map;
040    import java.util.Map.Entry;
041    
042    import org.apache.commons.io.input.BoundedInputStream;
043    import org.apache.commons.logging.Log;
044    import org.apache.commons.logging.LogFactory;
045    import org.apache.hadoop.classification.InterfaceAudience.Private;
046    import org.apache.hadoop.classification.InterfaceAudience.Public;
047    import org.apache.hadoop.classification.InterfaceStability.Evolving;
048    import org.apache.hadoop.conf.Configuration;
049    import org.apache.hadoop.fs.CreateFlag;
050    import org.apache.hadoop.fs.FSDataInputStream;
051    import org.apache.hadoop.fs.FSDataOutputStream;
052    import org.apache.hadoop.fs.FileContext;
053    import org.apache.hadoop.fs.Options;
054    import org.apache.hadoop.fs.Path;
055    import org.apache.hadoop.fs.permission.FsPermission;
056    import org.apache.hadoop.io.SecureIOUtils;
057    import org.apache.hadoop.io.Writable;
058    import org.apache.hadoop.io.file.tfile.TFile;
059    import org.apache.hadoop.security.UserGroupInformation;
060    import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
061    import org.apache.hadoop.yarn.api.records.ContainerId;
062    import org.apache.hadoop.yarn.conf.YarnConfiguration;
063    import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
064    import org.apache.hadoop.yarn.util.ConverterUtils;
065    
066    @Public
067    @Evolving
068    public class AggregatedLogFormat {
069    
070      private static final Log LOG = LogFactory.getLog(AggregatedLogFormat.class);
071      private static final LogKey APPLICATION_ACL_KEY = new LogKey("APPLICATION_ACL");
072      private static final LogKey APPLICATION_OWNER_KEY = new LogKey("APPLICATION_OWNER");
073      private static final LogKey VERSION_KEY = new LogKey("VERSION");
074      private static final Map<String, LogKey> RESERVED_KEYS;
075      //Maybe write out the retention policy.
076      //Maybe write out a list of containerLogs skipped by the retention policy.
077      private static final int VERSION = 1;
078    
079      /**
080       * Umask for the log file.
081       */
082      private static final FsPermission APP_LOG_FILE_UMASK = FsPermission
083          .createImmutable((short) (0640 ^ 0777));
084    
085    
086      static {
087        RESERVED_KEYS = new HashMap<String, AggregatedLogFormat.LogKey>();
088        RESERVED_KEYS.put(APPLICATION_ACL_KEY.toString(), APPLICATION_ACL_KEY);
089        RESERVED_KEYS.put(APPLICATION_OWNER_KEY.toString(), APPLICATION_OWNER_KEY);
090        RESERVED_KEYS.put(VERSION_KEY.toString(), VERSION_KEY);
091      }
092    
093      @Public
094      public static class LogKey implements Writable {
095    
096        private String keyString;
097    
098        public LogKey() {
099    
100        }
101    
102        public LogKey(ContainerId containerId) {
103          this.keyString = containerId.toString();
104        }
105    
106        public LogKey(String keyString) {
107          this.keyString = keyString;
108        }
109        
110        @Override
111        public int hashCode() {
112          return keyString == null ? 0 : keyString.hashCode();
113        }
114    
115        @Override
116        public boolean equals(Object obj) {
117          if (obj instanceof LogKey) {
118            LogKey other = (LogKey) obj;
119            if (this.keyString == null) {
120              return other.keyString == null;
121            }
122            return this.keyString.equals(other.keyString);
123          }
124          return false;
125        }
126    
127        @Private
128        @Override
129        public void write(DataOutput out) throws IOException {
130          out.writeUTF(this.keyString);
131        }
132    
133        @Private
134        @Override
135        public void readFields(DataInput in) throws IOException {
136          this.keyString = in.readUTF();
137        }
138    
139        @Override
140        public String toString() {
141          return this.keyString;
142        }
143      }
144    
145      @Private
146      public static class LogValue {
147    
148        private final List<String> rootLogDirs;
149        private final ContainerId containerId;
150        private final String user;
151        // TODO Maybe add a version string here. Instead of changing the version of
152        // the entire k-v format
153    
154        public LogValue(List<String> rootLogDirs, ContainerId containerId,
155            String user) {
156          this.rootLogDirs = new ArrayList<String>(rootLogDirs);
157          this.containerId = containerId;
158          this.user = user;
159    
160          // Ensure logs are processed in lexical order
161          Collections.sort(this.rootLogDirs);
162        }
163    
164        public void write(DataOutputStream out) throws IOException {
165          for (String rootLogDir : this.rootLogDirs) {
166            File appLogDir =
167                new File(rootLogDir, 
168                    ConverterUtils.toString(
169                        this.containerId.getApplicationAttemptId().
170                            getApplicationId())
171                    );
172            File containerLogDir =
173                new File(appLogDir, ConverterUtils.toString(this.containerId));
174    
175            if (!containerLogDir.isDirectory()) {
176              continue; // ContainerDir may have been deleted by the user.
177            }
178    
179            // Write out log files in lexical order
180            File[] logFiles = containerLogDir.listFiles();
181            Arrays.sort(logFiles);
182            for (File logFile : logFiles) {
183    
184              // Write the logFile Type
185              out.writeUTF(logFile.getName());
186    
187              // Write the log length as UTF so that it is printable
188              out.writeUTF(String.valueOf(logFile.length()));
189    
190              // Write the log itself
191              FileInputStream in = null;
192              try {
193                in = SecureIOUtils.openForRead(logFile, getUser(), null);
194                byte[] buf = new byte[65535];
195                int len = 0;
196                while ((len = in.read(buf)) != -1) {
197                  out.write(buf, 0, len);
198                }
199              } catch (IOException e) {
200                String message = "Error aggregating log file. Log file : "
201                    + logFile.getAbsolutePath() + e.getMessage(); 
202                LOG.error(message, e);
203                out.write(message.getBytes());
204              } finally {
205                if (in != null) {
206                  in.close();
207                }
208              }
209            }
210          }
211        }
212        
213        // Added for testing purpose.
214        public String getUser() {
215          return user;
216        }
217      }
218    
219      /**
220       * The writer that writes out the aggregated logs.
221       */
222      @Private
223      public static class LogWriter {
224    
225        private final FSDataOutputStream fsDataOStream;
226        private final TFile.Writer writer;
227    
228        public LogWriter(final Configuration conf, final Path remoteAppLogFile,
229            UserGroupInformation userUgi) throws IOException {
230          try {
231            this.fsDataOStream =
232                userUgi.doAs(new PrivilegedExceptionAction<FSDataOutputStream>() {
233                  @Override
234                  public FSDataOutputStream run() throws Exception {
235                    FileContext fc = FileContext.getFileContext(conf);
236                    fc.setUMask(APP_LOG_FILE_UMASK);
237                    return fc.create(
238                        remoteAppLogFile,
239                        EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
240                        new Options.CreateOpts[] {});
241                  }
242                });
243          } catch (InterruptedException e) {
244            throw new IOException(e);
245          }
246    
247          // Keys are not sorted: null arg
248          // 256KB minBlockSize : Expected log size for each container too
249          this.writer =
250              new TFile.Writer(this.fsDataOStream, 256 * 1024, conf.get(
251                  YarnConfiguration.NM_LOG_AGG_COMPRESSION_TYPE,
252                  YarnConfiguration.DEFAULT_NM_LOG_AGG_COMPRESSION_TYPE), null, conf);
253          //Write the version string
254          writeVersion();
255        }
256    
257        private void writeVersion() throws IOException {
258          DataOutputStream out = this.writer.prepareAppendKey(-1);
259          VERSION_KEY.write(out);
260          out.close();
261          out = this.writer.prepareAppendValue(-1);
262          out.writeInt(VERSION);
263          out.close();
264        }
265    
266        public void writeApplicationOwner(String user) throws IOException {
267          DataOutputStream out = this.writer.prepareAppendKey(-1);
268          APPLICATION_OWNER_KEY.write(out);
269          out.close();
270          out = this.writer.prepareAppendValue(-1);
271          out.writeUTF(user);
272          out.close();
273        }
274    
275        public void writeApplicationACLs(Map<ApplicationAccessType, String> appAcls)
276            throws IOException {
277          DataOutputStream out = this.writer.prepareAppendKey(-1);
278          APPLICATION_ACL_KEY.write(out);
279          out.close();
280          out = this.writer.prepareAppendValue(-1);
281          for (Entry<ApplicationAccessType, String> entry : appAcls.entrySet()) {
282            out.writeUTF(entry.getKey().toString());
283            out.writeUTF(entry.getValue());
284          }
285          out.close();
286        }
287    
288        public void append(LogKey logKey, LogValue logValue) throws IOException {
289          DataOutputStream out = this.writer.prepareAppendKey(-1);
290          logKey.write(out);
291          out.close();
292          out = this.writer.prepareAppendValue(-1);
293          logValue.write(out);
294          out.close();
295        }
296    
297        public void closeWriter() {
298          try {
299            this.writer.close();
300          } catch (IOException e) {
301            LOG.warn("Exception closing writer", e);
302          }
303          try {
304            this.fsDataOStream.close();
305          } catch (IOException e) {
306            LOG.warn("Exception closing output-stream", e);
307          }
308        }
309      }
310    
311      @Public
312      @Evolving
313      public static class LogReader {
314    
315        private final FSDataInputStream fsDataIStream;
316        private final TFile.Reader.Scanner scanner;
317        private final TFile.Reader reader;
318    
319        public LogReader(Configuration conf, Path remoteAppLogFile)
320            throws IOException {
321          FileContext fileContext = FileContext.getFileContext(conf);
322          this.fsDataIStream = fileContext.open(remoteAppLogFile);
323          reader =
324              new TFile.Reader(this.fsDataIStream, fileContext.getFileStatus(
325                  remoteAppLogFile).getLen(), conf);
326          this.scanner = reader.createScanner();
327        }
328    
329        private boolean atBeginning = true;
330    
331        /**
332         * Returns the owner of the application.
333         * 
334         * @return the application owner.
335         * @throws IOException
336         */
337        public String getApplicationOwner() throws IOException {
338          TFile.Reader.Scanner ownerScanner = reader.createScanner();
339          LogKey key = new LogKey();
340          while (!ownerScanner.atEnd()) {
341            TFile.Reader.Scanner.Entry entry = ownerScanner.entry();
342            key.readFields(entry.getKeyStream());
343            if (key.toString().equals(APPLICATION_OWNER_KEY.toString())) {
344              DataInputStream valueStream = entry.getValueStream();
345              return valueStream.readUTF();
346            }
347            ownerScanner.advance();
348          }
349          return null;
350        }
351    
352        /**
353         * Returns ACLs for the application. An empty map is returned if no ACLs are
354         * found.
355         * 
356         * @return a map of the Application ACLs.
357         * @throws IOException
358         */
359        public Map<ApplicationAccessType, String> getApplicationAcls()
360            throws IOException {
361          // TODO Seek directly to the key once a comparator is specified.
362          TFile.Reader.Scanner aclScanner = reader.createScanner();
363          LogKey key = new LogKey();
364          Map<ApplicationAccessType, String> acls =
365              new HashMap<ApplicationAccessType, String>();
366          while (!aclScanner.atEnd()) {
367            TFile.Reader.Scanner.Entry entry = aclScanner.entry();
368            key.readFields(entry.getKeyStream());
369            if (key.toString().equals(APPLICATION_ACL_KEY.toString())) {
370              DataInputStream valueStream = entry.getValueStream();
371              while (true) {
372                String appAccessOp = null;
373                String aclString = null;
374                try {
375                  appAccessOp = valueStream.readUTF();
376                } catch (EOFException e) {
377                  // Valid end of stream.
378                  break;
379                }
380                try {
381                  aclString = valueStream.readUTF();
382                } catch (EOFException e) {
383                  throw new YarnRuntimeException("Error reading ACLs", e);
384                }
385                acls.put(ApplicationAccessType.valueOf(appAccessOp), aclString);
386              }
387    
388            }
389            aclScanner.advance();
390          }
391          return acls;
392        }
393        
394        /**
395         * Read the next key and return the value-stream.
396         * 
397         * @param key
398         * @return the valueStream if there are more keys or null otherwise.
399         * @throws IOException
400         */
401        public DataInputStream next(LogKey key) throws IOException {
402          if (!this.atBeginning) {
403            this.scanner.advance();
404          } else {
405            this.atBeginning = false;
406          }
407          if (this.scanner.atEnd()) {
408            return null;
409          }
410          TFile.Reader.Scanner.Entry entry = this.scanner.entry();
411          key.readFields(entry.getKeyStream());
412          // Skip META keys
413          if (RESERVED_KEYS.containsKey(key.toString())) {
414            return next(key);
415          }
416          DataInputStream valueStream = entry.getValueStream();
417          return valueStream;
418        }
419    
420        /**
421         * Get a ContainerLogsReader to read the logs for
422         * the specified container.
423         *
424         * @param containerId
425         * @return object to read the container's logs or null if the
426         *         logs could not be found
427         * @throws IOException
428         */
429        @Private
430        public ContainerLogsReader getContainerLogsReader(
431            ContainerId containerId) throws IOException {
432          ContainerLogsReader logReader = null;
433    
434          final LogKey containerKey = new LogKey(containerId);
435          LogKey key = new LogKey();
436          DataInputStream valueStream = next(key);
437          while (valueStream != null && !key.equals(containerKey)) {
438            valueStream = next(key);
439          }
440    
441          if (valueStream != null) {
442            logReader = new ContainerLogsReader(valueStream);
443          }
444    
445          return logReader;
446        }
447    
448        //TODO  Change Log format and interfaces to be containerId specific.
449        // Avoid returning completeValueStreams.
450    //    public List<String> getTypesForContainer(DataInputStream valueStream){}
451    //    
452    //    /**
453    //     * @param valueStream
454    //     *          The Log stream for the container.
455    //     * @param fileType
456    //     *          the log type required.
457    //     * @return An InputStreamReader for the required log type or null if the
458    //     *         type is not found.
459    //     * @throws IOException
460    //     */
461    //    public InputStreamReader getLogStreamForType(DataInputStream valueStream,
462    //        String fileType) throws IOException {
463    //      valueStream.reset();
464    //      try {
465    //        while (true) {
466    //          String ft = valueStream.readUTF();
467    //          String fileLengthStr = valueStream.readUTF();
468    //          long fileLength = Long.parseLong(fileLengthStr);
469    //          if (ft.equals(fileType)) {
470    //            BoundedInputStream bis =
471    //                new BoundedInputStream(valueStream, fileLength);
472    //            return new InputStreamReader(bis);
473    //          } else {
474    //            long totalSkipped = 0;
475    //            long currSkipped = 0;
476    //            while (currSkipped != -1 && totalSkipped < fileLength) {
477    //              currSkipped = valueStream.skip(fileLength - totalSkipped);
478    //              totalSkipped += currSkipped;
479    //            }
480    //            // TODO Verify skip behaviour.
481    //            if (currSkipped == -1) {
482    //              return null;
483    //            }
484    //          }
485    //        }
486    //      } catch (EOFException e) {
487    //        return null;
488    //      }
489    //    }
490    
491        /**
492         * Writes all logs for a single container to the provided writer.
493         * @param valueStream
494         * @param writer
495         * @throws IOException
496         */
497        public static void readAcontainerLogs(DataInputStream valueStream,
498            Writer writer) throws IOException {
499          int bufferSize = 65536;
500          char[] cbuf = new char[bufferSize];
501          String fileType;
502          String fileLengthStr;
503          long fileLength;
504    
505          while (true) {
506            try {
507              fileType = valueStream.readUTF();
508            } catch (EOFException e) {
509              // EndOfFile
510              return;
511            }
512            fileLengthStr = valueStream.readUTF();
513            fileLength = Long.parseLong(fileLengthStr);
514            writer.write("\n\nLogType:");
515            writer.write(fileType);
516            writer.write("\nLogLength:");
517            writer.write(fileLengthStr);
518            writer.write("\nLog Contents:\n");
519            // ByteLevel
520            BoundedInputStream bis =
521                new BoundedInputStream(valueStream, fileLength);
522            InputStreamReader reader = new InputStreamReader(bis);
523            int currentRead = 0;
524            int totalRead = 0;
525            while ((currentRead = reader.read(cbuf, 0, bufferSize)) != -1) {
526              writer.write(cbuf, 0, currentRead);
527              totalRead += currentRead;
528            }
529          }
530        }
531    
532        /**
533         * Keep calling this till you get a {@link EOFException} for getting logs of
534         * all types for a single container.
535         * 
536         * @param valueStream
537         * @param out
538         * @throws IOException
539         */
540        public static void readAContainerLogsForALogType(
541            DataInputStream valueStream, PrintStream out)
542              throws IOException {
543    
544          byte[] buf = new byte[65535];
545    
546          String fileType = valueStream.readUTF();
547          String fileLengthStr = valueStream.readUTF();
548          long fileLength = Long.parseLong(fileLengthStr);
549          out.print("LogType: ");
550          out.println(fileType);
551          out.print("LogLength: ");
552          out.println(fileLengthStr);
553          out.println("Log Contents:");
554    
555          int curRead = 0;
556          long pendingRead = fileLength - curRead;
557          int toRead =
558                    pendingRead > buf.length ? buf.length : (int) pendingRead;
559          int len = valueStream.read(buf, 0, toRead);
560          while (len != -1 && curRead < fileLength) {
561            out.write(buf, 0, len);
562            curRead += len;
563    
564            pendingRead = fileLength - curRead;
565            toRead =
566                      pendingRead > buf.length ? buf.length : (int) pendingRead;
567            len = valueStream.read(buf, 0, toRead);
568          }
569          out.println("");
570        }
571    
572        public void close() throws IOException {
573          this.scanner.close();
574          this.fsDataIStream.close();
575        }
576      }
577    
578      @Private
579      public static class ContainerLogsReader {
580        private DataInputStream valueStream;
581        private String currentLogType = null;
582        private long currentLogLength = 0;
583        private BoundedInputStream currentLogData = null;
584        private InputStreamReader currentLogISR;
585    
586        public ContainerLogsReader(DataInputStream stream) {
587          valueStream = stream;
588        }
589    
590        public String nextLog() throws IOException {
591          if (currentLogData != null && currentLogLength > 0) {
592            // seek to the end of the current log, relying on BoundedInputStream
593            // to prevent seeking past the end of the current log
594            do {
595              if (currentLogData.skip(currentLogLength) < 0) {
596                break;
597              }
598            } while (currentLogData.read() != -1);
599          }
600    
601          currentLogType = null;
602          currentLogLength = 0;
603          currentLogData = null;
604          currentLogISR = null;
605    
606          try {
607            String logType = valueStream.readUTF();
608            String logLengthStr = valueStream.readUTF();
609            currentLogLength = Long.parseLong(logLengthStr);
610            currentLogData =
611                new BoundedInputStream(valueStream, currentLogLength);
612            currentLogData.setPropagateClose(false);
613            currentLogISR = new InputStreamReader(currentLogData);
614            currentLogType = logType;
615          } catch (EOFException e) {
616          }
617    
618          return currentLogType;
619        }
620    
621        public String getCurrentLogType() {
622          return currentLogType;
623        }
624    
625        public long getCurrentLogLength() {
626          return currentLogLength;
627        }
628    
629        public long skip(long n) throws IOException {
630          return currentLogData.skip(n);
631        }
632    
633        public int read(byte[] buf, int off, int len) throws IOException {
634          return currentLogData.read(buf, off, len);
635        }
636    
637        public int read(char[] buf, int off, int len) throws IOException {
638          return currentLogISR.read(buf, off, len);
639        }
640      }
641    }