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

    https://github.com/apache/spark/pull/12435#discussion_r60228316
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala
 ---
    @@ -0,0 +1,255 @@
    +/*
    + * 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.{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 action the file action. Must be either "add" or "delete".
    + */
    +case class SinkFileStatus(path: String, size: Long, action: String)
    +
    +/**
    + * 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. The user
    +   * should set a reasonable `fileCleanupDelayMs`. We will wait until then 
so that the compaction
    +   * file is guaranteed to be visible for all readers
    +   */
    +  private val fileCleanupDelayMs = 
sqlContext.getConf(SQLConf.FILE_SINK_LOG_CLEANUP_DELAY)
    --- End diff --
    
    all AWS S3 endpoints now implement create consistency: if a new object is 
created, then a GET made directly on it will return that object.
    
    what can take time to appear is the aggregate file in an ls of the parent 
"directory" —that's really a wild card match on the path. If the processes 
can determine the final name of the compaction file, they can look for that 
file directly (getFileStatus() should suffice, open() even better). If the 
compact file isn't found, they can look for the non-aggregate files. All that 
should be required is the aggregate file fully written (with a close() at the 
end of output operation *which doesn't discard any raised exception*), before 
deleting the original files. Adding a minor delay is a low-harm feature, but 
having a direct check for the aggregate file is something which should be done 
first



---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to