Github user tedyu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12435#discussion_r60493854
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala
 ---
    @@ -0,0 +1,278 @@
    +/*
    + * 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.sql.execution.streaming
    +
    +import java.io.IOException
    +import java.nio.charset.StandardCharsets.UTF_8
    +
    +import org.apache.hadoop.fs.{FileStatus, Path, PathFilter}
    +import org.json4s.NoTypeHints
    +import org.json4s.jackson.Serialization
    +import org.json4s.jackson.Serialization.{read, write}
    +
    +import org.apache.spark.sql.SQLContext
    +import org.apache.spark.sql.internal.SQLConf
    +
    +/**
    + * The status of a file outputted by [[FileStreamSink]]. A file is visible 
only if it appears in
    + * the sink log and its action is not "delete".
    + *
    + * @param path the file path.
    + * @param size the file size.
    + * @param isDir whether this file is a directory.
    + * @param modificationTime the file last modification time.
    + * @param blockReplication the block replication.
    + * @param blockSize the block size.
    + * @param action the file action. Must be either "add" or "delete".
    + */
    +case class SinkFileStatus(
    +    path: String,
    +    size: Long,
    +    isDir: Boolean,
    +    modificationTime: Long,
    +    blockReplication: Int,
    +    blockSize: Long,
    +    action: String) {
    +
    +  def toFileStatus: FileStatus = {
    +    new FileStatus(size, isDir, blockReplication, blockSize, 
modificationTime, new Path(path))
    +  }
    +}
    +
    +/**
    + * A special log for [[FileStreamSink]]. It will write one log file for 
each batch. The first line
    + * of the log file is the version number, and there are multiple JSON 
lines following. Each JSON
    + * line is a JSON format of [[SinkFileStatus]].
    + *
    + * As reading from many small files is usually pretty slow, 
[[FileStreamSinkLog]] will compact log
    + * files every "spark.sql.sink.file.log.compactLen" batches into a big 
file. When doing a
    + * compaction, it will read all old log files and merge them with the new 
batch. During the
    + * compaction, it will also delete the files that are deleted (marked by 
[[SinkFileStatus.action]]).
    + * When the reader uses `allFiles` to list all files, this method only 
returns the visible files
    + * (drops the deleted files).
    + */
    +class FileStreamSinkLog(sqlContext: SQLContext, path: String)
    +  extends HDFSMetadataLog[Seq[SinkFileStatus]](sqlContext, path) {
    +
    +  import FileStreamSinkLog._
    +
    +  private implicit val formats = Serialization.formats(NoTypeHints)
    +
    +  /**
    +   * If we delete the old files after compaction at once, there is a race 
condition in S3: other
    +   * processes may see the old files are deleted but still cannot see the 
compaction file using
    +   * "list". The `allFiles` handles this by looking for the next 
compaction file directly, however,
    +   * a live lock may happen if the compaction happens too frequently: one 
processing keeps deleting
    +   * old files while another one keeps retrying. Setting a reasonable 
cleanup delay could avoid it.
    +   */
    +  private val fileCleanupDelayMs = 
sqlContext.getConf(SQLConf.FILE_SINK_LOG_CLEANUP_DELAY)
    +
    +  private val isDeletingExpiredLog = 
sqlContext.getConf(SQLConf.FILE_SINK_LOG_DELETION)
    +
    +  private val compactInterval = 
sqlContext.getConf(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL)
    +  require(compactInterval > 0,
    +    s"Please set ${SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key} (was 
$compactInterval) " +
    +      "to a positive value.")
    +
    +  override def batchIdToPath(batchId: Long): Path = {
    +    if (isCompactionBatch(batchId, compactInterval)) {
    +      new Path(metadataPath, s"$batchId$COMPACT_FILE_SUFFIX")
    +    } else {
    +      new Path(metadataPath, batchId.toString)
    +    }
    +  }
    +
    +  override def pathToBatchId(path: Path): Long = {
    +    getBatchIdFromFileName(path.getName)
    +  }
    +
    +  override def isBatchFile(path: Path): Boolean = {
    +    try {
    +      getBatchIdFromFileName(path.getName)
    +      true
    +    } catch {
    +      case _: NumberFormatException => false
    +    }
    +  }
    +
    +  override def serialize(logData: Seq[SinkFileStatus]): Array[Byte] = {
    +    (VERSION +: logData.map(write(_))).mkString("\n").getBytes(UTF_8)
    +  }
    +
    +  override def deserialize(bytes: Array[Byte]): Seq[SinkFileStatus] = {
    +    val lines = new String(bytes, UTF_8).split("\n")
    +    if (lines.length == 0) {
    +      throw new IllegalStateException("Incomplete log file")
    +    }
    +    val version = lines(0)
    +    if (version != VERSION) {
    --- End diff --
    
    Should this be 'version > VERSION' ?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to