Return-Path: X-Original-To: apmail-spark-dev-archive@minotaur.apache.org Delivered-To: apmail-spark-dev-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 631601043E for ; Tue, 4 Mar 2014 20:06:43 +0000 (UTC) Received: (qmail 26760 invoked by uid 500); 4 Mar 2014 20:06:42 -0000 Delivered-To: apmail-spark-dev-archive@spark.apache.org Received: (qmail 26682 invoked by uid 500); 4 Mar 2014 20:06:41 -0000 Mailing-List: contact dev-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@spark.apache.org Delivered-To: mailing list dev@spark.apache.org Received: (qmail 26488 invoked by uid 99); 4 Mar 2014 20:06:41 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 Mar 2014 20:06:41 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 1254F935E11; Tue, 4 Mar 2014 20:06:41 +0000 (UTC) From: pwendell To: dev@spark.apache.org Reply-To: dev@spark.apache.org References: In-Reply-To: Subject: [GitHub] spark pull request: [WIP] [SPARK-1132] Persisting Web UI through r... Content-Type: text/plain Message-Id: <20140304200641.1254F935E11@tyr.zones.apache.org> Date: Tue, 4 Mar 2014 20:06:41 +0000 (UTC) Github user pwendell commented on a diff in the pull request: https://github.com/apache/spark/pull/42#discussion_r10272169 --- Diff: core/src/main/scala/org/apache/spark/util/FileLogger.scala --- @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io._ +import java.text.SimpleDateFormat +import java.net.URI +import java.util.Date + +import org.apache.spark.Logging +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.hadoop.fs.{FSDataOutputStream, Path, FileSystem} + +/** + * A generic class for logging information to file. + * + * @param logBaseDir Path to the directory in which files are logged + * @param name An identifier of each FileLogger instance + * @param overwriteExistingFiles Whether to overwrite existing files + */ +class FileLogger( + logBaseDir: String, + name: String = String.valueOf(System.currentTimeMillis()), + overwriteExistingFiles: Boolean = true) + extends Logging { + + private val logDir = logBaseDir.stripSuffix("/") + "/" + name + private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private var fileIndex = 0 + + // Only defined if the file system scheme uses the Hadoop API + private var hadoopDataStream: Option[FSDataOutputStream] = None + private var hadoopFileSystem: Option[FileSystem] = None + + private var writer: Option[PrintWriter] = { + createLogDir() + createWriter() + } + + /** Create a logging directory with the given path */ + private def createLogDir() { + val dir = new File(logDir) + if (dir.exists) { + logWarning("Logging directory already exists: " + logDir) + } + if (!dir.exists && !dir.mkdirs()) { + // Logger should throw a exception rather than continue to construct this object + throw new IOException("Error in creating log directory:" + logDir) + } + } + + /** + * Create a new writer for the file identified by the given path. + * + * File systems currently supported include HDFS, S3, and the local file system. + * The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + * Therefore, for local files, use FileOutputStream instead. + */ + private def createWriter(): Option[PrintWriter] = { + val logPath = logDir + "/" + fileIndex + val uri = new URI(logPath) + + val dataStream = uri.getScheme match { + case "hdfs" | "s3" => + val fs = hadoopFileSystem.getOrElse { + val conf = SparkHadoopUtil.get.newConfiguration() + hadoopFileSystem = Some(FileSystem.get(uri, conf)) + hadoopFileSystem.get + } + val path = new Path(logPath) + hadoopDataStream = Some(fs.create(path, overwriteExistingFiles)) + hadoopDataStream.get + + case "file" | null => + // Second parameter is whether to append + new FileOutputStream(logPath, !overwriteExistingFiles) + + case unsupportedScheme => + throw new UnsupportedOperationException("File system scheme %s is not supported!" + .format(unsupportedScheme)) + } + + val bufferedStream = new BufferedOutputStream(dataStream) --- End diff -- Here it would probably be good to make the size of the buffer configurable. And maybe default to 100 KB or something. I think the default is around 8KB for this stream. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastructure@apache.org or file a JIRA ticket with INFRA. ---