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 }