aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
blob: ddba3ccb1ba50173f09f133d4ccac3e061a1e502 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
/*
* 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.{FileNotFoundException, IOException}
import java.nio.ByteBuffer
import java.util.{ConcurrentModificationException, EnumSet, UUID}

import scala.reflect.ClassTag

import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._
import org.apache.hadoop.fs.permission.FsPermission

import org.apache.spark.internal.Logging
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.serializer.JavaSerializer
import org.apache.spark.sql.SparkSession


/**
 * A [[MetadataLog]] implementation based on HDFS. [[HDFSMetadataLog]] uses the specified `path`
 * as the metadata storage.
 *
 * When writing a new batch, [[HDFSMetadataLog]] will firstly write to a temp file and then rename
 * it to the final batch file. If the rename step fails, there must be multiple writers and only
 * one of them will succeed and the others will fail.
 *
 * Note: [[HDFSMetadataLog]] doesn't support S3-like file systems as they don't guarantee listing
 * files in a directory always shows the latest files.
 */
class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String)
  extends MetadataLog[T]
  with Logging {

  import HDFSMetadataLog._

  val metadataPath = new Path(path)
  protected val fileManager = createFileManager()

  if (!fileManager.exists(metadataPath)) {
    fileManager.mkdirs(metadataPath)
  }

  /**
   * A `PathFilter` to filter only batch files
   */
  private val batchFilesFilter = new PathFilter {
    override def accept(path: Path): Boolean = isBatchFile(path)
  }

  private val serializer = new JavaSerializer(sparkSession.sparkContext.conf).newInstance()

  protected def batchIdToPath(batchId: Long): Path = {
    new Path(metadataPath, batchId.toString)
  }

  protected def pathToBatchId(path: Path) = {
    path.getName.toLong
  }

  protected def isBatchFile(path: Path) = {
    try {
      path.getName.toLong
      true
    } catch {
      case _: NumberFormatException => false
    }
  }

  protected def serialize(metadata: T): Array[Byte] = {
    JavaUtils.bufferToArray(serializer.serialize(metadata))
  }

  protected def deserialize(bytes: Array[Byte]): T = {
    serializer.deserialize[T](ByteBuffer.wrap(bytes))
  }

  override def add(batchId: Long, metadata: T): Boolean = {
    get(batchId).map(_ => false).getOrElse {
      // Only write metadata when the batch has not yet been written.
      try {
        writeBatch(batchId, serialize(metadata))
        true
      } catch {
        case e: IOException if "java.lang.InterruptedException" == e.getMessage =>
          // create may convert InterruptedException to IOException. Let's convert it back to
          // InterruptedException so that this failure won't crash StreamExecution
          throw new InterruptedException("Creating file is interrupted")
      }
    }
  }

  /**
   * Write a batch to a temp file then rename it to the batch file.
   *
   * There may be multiple [[HDFSMetadataLog]] using the same metadata path. Although it is not a
   * valid behavior, we still need to prevent it from destroying the files.
   */
  private def writeBatch(batchId: Long, bytes: Array[Byte]): Unit = {
    // Use nextId to create a temp file
    var nextId = 0
    while (true) {
      val tempPath = new Path(metadataPath, s".${UUID.randomUUID.toString}.tmp")
      try {
        val output = fileManager.create(tempPath)
        try {
          output.write(bytes)
        } finally {
          output.close()
        }
        try {
          // Try to commit the batch
          // It will fail if there is an existing file (someone has committed the batch)
          logDebug(s"Attempting to write log #${batchIdToPath(batchId)}")
          fileManager.rename(tempPath, batchIdToPath(batchId))
          return
        } catch {
          case e: IOException if isFileAlreadyExistsException(e) =>
            // If "rename" fails, it means some other "HDFSMetadataLog" has committed the batch.
            // So throw an exception to tell the user this is not a valid behavior.
            throw new ConcurrentModificationException(
              s"Multiple HDFSMetadataLog are using $path", e)
          case e: FileNotFoundException =>
            // Sometimes, "create" will succeed when multiple writers are calling it at the same
            // time. However, only one writer can call "rename" successfully, others will get
            // FileNotFoundException because the first writer has removed it.
            throw new ConcurrentModificationException(
              s"Multiple HDFSMetadataLog are using $path", e)
        }
      } catch {
        case e: IOException if isFileAlreadyExistsException(e) =>
          // Failed to create "tempPath". There are two cases:
          // 1. Someone is creating "tempPath" too.
          // 2. This is a restart. "tempPath" has already been created but not moved to the final
          // batch file (not committed).
          //
          // For both cases, the batch has not yet been committed. So we can retry it.
          //
          // Note: there is a potential risk here: if HDFSMetadataLog A is running, people can use
          // the same metadata path to create "HDFSMetadataLog" and fail A. However, this is not a
          // big problem because it requires the attacker must have the permission to write the
          // metadata path. In addition, the old Streaming also have this issue, people can create
          // malicious checkpoint files to crash a Streaming application too.
          nextId += 1
      } finally {
        fileManager.delete(tempPath)
      }
    }
  }

  private def isFileAlreadyExistsException(e: IOException): Boolean = {
    e.isInstanceOf[FileAlreadyExistsException] ||
      // Old Hadoop versions don't throw FileAlreadyExistsException. Although it's fixed in
      // HADOOP-9361, we still need to support old Hadoop versions.
      (e.getMessage != null && e.getMessage.startsWith("File already exists: "))
  }

  override def get(batchId: Long): Option[T] = {
    val batchMetadataFile = batchIdToPath(batchId)
    if (fileManager.exists(batchMetadataFile)) {
      val input = fileManager.open(batchMetadataFile)
      val bytes = IOUtils.toByteArray(input)
      Some(deserialize(bytes))
    } else {
      logDebug(s"Unable to find batch $batchMetadataFile")
      None
    }
  }

  override def get(startId: Option[Long], endId: Option[Long]): Array[(Long, T)] = {
    val files = fileManager.list(metadataPath, batchFilesFilter)
    val batchIds = files
      .map(f => pathToBatchId(f.getPath))
      .filter { batchId =>
        (endId.isEmpty || batchId <= endId.get) && (startId.isEmpty || batchId >= startId.get)
    }
    batchIds.sorted.map(batchId => (batchId, get(batchId))).filter(_._2.isDefined).map {
      case (batchId, metadataOption) =>
        (batchId, metadataOption.get)
    }
  }

  override def getLatest(): Option[(Long, T)] = {
    val batchIds = fileManager.list(metadataPath, batchFilesFilter)
      .map(f => pathToBatchId(f.getPath))
      .sorted
      .reverse
    for (batchId <- batchIds) {
      val batch = get(batchId)
      if (batch.isDefined) {
        return Some((batchId, batch.get))
      }
    }
    None
  }

  private def createFileManager(): FileManager = {
    val hadoopConf = new Configuration(sparkSession.sessionState.hadoopConf)
    try {
      new FileContextManager(metadataPath, hadoopConf)
    } catch {
      case e: UnsupportedFileSystemException =>
        logWarning("Could not use FileContext API for managing metadata log file. The log may be" +
          "inconsistent under failures.", e)
        new FileSystemManager(metadataPath, hadoopConf)
    }
  }
}

object HDFSMetadataLog {

  /** A simple trait to abstract out the file management operations needed by HDFSMetadataLog. */
  trait FileManager {

    /** List the files in a path that matches a filter. */
    def list(path: Path, filter: PathFilter): Array[FileStatus]

    /** Make directory at the give path and all its parent directories as needed. */
    def mkdirs(path: Path): Unit

    /** Whether path exists */
    def exists(path: Path): Boolean

    /** Open a file for reading, or throw exception if it does not exist. */
    def open(path: Path): FSDataInputStream

    /** Create path, or throw exception if it already exists */
    def create(path: Path): FSDataOutputStream

    /**
     * Atomically rename path, or throw exception if it cannot be done.
     * Should throw FileNotFoundException if srcPath does not exist.
     * Should throw FileAlreadyExistsException if destPath already exists.
     */
    def rename(srcPath: Path, destPath: Path): Unit

    /** Recursively delete a path if it exists. Should not throw exception if file doesn't exist. */
    def delete(path: Path): Unit
  }

  /**
   * Default implementation of FileManager using newer FileContext API.
   */
  class FileContextManager(path: Path, hadoopConf: Configuration) extends FileManager {
    private val fc = if (path.toUri.getScheme == null) {
      FileContext.getFileContext(hadoopConf)
    } else {
      FileContext.getFileContext(path.toUri, hadoopConf)
    }

    override def list(path: Path, filter: PathFilter): Array[FileStatus] = {
      fc.util.listStatus(path, filter)
    }

    override def rename(srcPath: Path, destPath: Path): Unit = {
      fc.rename(srcPath, destPath)
    }

    override def mkdirs(path: Path): Unit = {
      fc.mkdir(path, FsPermission.getDirDefault, true)
    }

    override def open(path: Path): FSDataInputStream = {
      fc.open(path)
    }

    override def create(path: Path): FSDataOutputStream = {
      fc.create(path, EnumSet.of(CreateFlag.CREATE))
    }

    override def exists(path: Path): Boolean = {
      fc.util().exists(path)
    }

    override def delete(path: Path): Unit = {
      try {
        fc.delete(path, true)
      } catch {
        case e: FileNotFoundException =>
        // ignore if file has already been deleted
      }
    }
  }

  /**
   * Implementation of FileManager using older FileSystem API. Note that this implementation
   * cannot provide atomic renaming of paths, hence can lead to consistency issues. This
   * should be used only as a backup option, when FileContextManager cannot be used.
   */
  class FileSystemManager(path: Path, hadoopConf: Configuration) extends FileManager {
    private val fs = path.getFileSystem(hadoopConf)

    override def list(path: Path, filter: PathFilter): Array[FileStatus] = {
      fs.listStatus(path, filter)
    }

    /**
     * Rename a path. Note that this implementation is not atomic.
     * @throws FileNotFoundException if source path does not exist.
     * @throws FileAlreadyExistsException if destination path already exists.
     * @throws IOException if renaming fails for some unknown reason.
     */
    override def rename(srcPath: Path, destPath: Path): Unit = {
      if (!fs.exists(srcPath)) {
        throw new FileNotFoundException(s"Source path does not exist: $srcPath")
      }
      if (fs.exists(destPath)) {
        throw new FileAlreadyExistsException(s"Destination path already exists: $destPath")
      }
      if (!fs.rename(srcPath, destPath)) {
        throw new IOException(s"Failed to rename $srcPath to $destPath")
      }
    }

    override def mkdirs(path: Path): Unit = {
      fs.mkdirs(path, FsPermission.getDirDefault)
    }

    override def open(path: Path): FSDataInputStream = {
      fs.open(path)
    }

    override def create(path: Path): FSDataOutputStream = {
      fs.create(path, false)
    }

    override def exists(path: Path): Boolean = {
      fs.exists(path)
    }

    override def delete(path: Path): Unit = {
      try {
        fs.delete(path, true)
      } catch {
        case e: FileNotFoundException =>
          // ignore if file has already been deleted
      }
    }
  }
}