You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by vanzin <gi...@git.apache.org> on 2017/12/18 21:29:46 UTC

[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

GitHub user vanzin opened a pull request:

    https://github.com/apache/spark/pull/20011

    [SPARK-20654][core] Add config to limit disk usage of the history server.

    This change adds a new configuration option and support code that limits
    how much disk space the SHS will use. The default value is pretty generous
    so that applications will, hopefully, only rarely need to be replayed
    because of their disk stored being evicted.
    
    This works by keeping track of how much data each application is using.
    Also, because it's not possible to know, before replaying, how much space
    will be needed, it's possible that usage will exceed the configured limit
    temporarily. The code uses the concept of a "lease" to try to limit how
    much the SHS will exceed the limit in those cases.
    
    Active UIs are also tracked, so they're never deleted. This works in
    tandem with the existing option of how many active UIs are loaded; because
    unused UIs will be unloaded, their disk stores will also become candidates
    for deletion. If the data is not deleted, though, re-loading the UI is
    pretty quick.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/vanzin/spark SPARK-20654

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/20011.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #20011
    
----
commit 8b43f4d2c06ab0a9916113fc216711573db367e7
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2017-04-25T19:59:42Z

    [SPARK-20654][core] Add config to limit disk usage of the history server.
    
    This change adds a new configuration option and support code that limits
    how much disk space the SHS will use. The default value is pretty generous
    so that applications will, hopefully, only rarely need to be replayed
    because of their disk stored being evicted.
    
    This works by keeping track of how much data each application is using.
    Also, because it's not possible to know, before replaying, how much space
    will be needed, it's possible that usage will exceed the configured limit
    temporarily. The code uses the concept of a "lease" to try to limit how
    much the SHS will exceed the limit in those cases.
    
    Active UIs are also tracked, so they're never deleted. This works in
    tandem with the existing option of how many active UIs are loaded; because
    unused UIs will be unloaded, their disk stores will also become candidates
    for deletion. If the data is not deleted, though, re-loading the UI is
    pretty quick.

----


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158558207
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/DiskStoreManager.scala ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +import java.nio.file.Files
    +import java.nio.file.attribute.PosixFilePermissions
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.{HashMap, ListBuffer}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.status.KVUtils._
    +import org.apache.spark.util.{Clock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +/**
    + * A class used to keep track of disk usage by the SHS, allowing application data to be deleted
    + * from disk when usage exceeds a configurable threshold.
    + *
    + * The goal of the class is not to guarantee that usage will never exceed the threshold; because of
    + * how application data is written, disk usage may temporarily go higher. But, eventually, it
    + * should fall back under the threshold.
    + *
    + * @param conf Spark configuration.
    + * @param path Path where to store application data.
    + * @param listing The listing store, used to persist usage data.
    + * @param clock Clock instance to use.
    + */
    +private class DiskStoreManager(
    +    conf: SparkConf,
    +    path: File,
    +    listing: KVStore,
    +    clock: Clock) extends Logging {
    +
    +  import config._
    +
    +  private val appStoreDir = new File(path, "apps")
    +  if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).")
    +  }
    +
    +  private val tmpStoreDir = new File(path, "temp")
    +  if (!tmpStoreDir.isDirectory() && !tmpStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create temp directory ($tmpStoreDir).")
    +  }
    +
    +  private val maxUsage = conf.get(MAX_LOCAL_DISK_USAGE)
    +  private val currentUsage = new AtomicLong(0L)
    +  private val active = new HashMap[(String, Option[String]), Long]()
    +
    +  def initialize(): Unit = {
    +    updateUsage(sizeOf(appStoreDir))
    +
    +    // Clean up any temporary stores during start up. This assumes that they're leftover from other
    +    // instances and are not useful.
    +    tmpStoreDir.listFiles().foreach(FileUtils.deleteQuietly)
    +
    +    // Go through the recorded store directories and remove any that may have been removed by
    +    // external code.
    +    val orphans = listing.view(classOf[ApplicationStoreInfo]).asScala.filter { info =>
    +      !new File(info.path).exists()
    +    }.toSeq
    +
    +    orphans.foreach { info =>
    +      listing.delete(info.getClass(), info.path)
    +    }
    +  }
    +
    +  /**
    +   * Lease some space from the store. The leased space is calculated as a fraction of the given
    +   * event log size; this is an approximation, and doesn't mean the application store cannot
    +   * outgrow the lease.
    +   *
    +   * If there's not enough space for the lease, other applications might be evicted to make room.
    +   * This method always returns a lease, meaning that it's possible for local disk usage to grow
    +   * past the configured threshold if there aren't enough idle applications to evict.
    +   *
    +   * While the lease is active, the data is written to a temporary location, so `openStore()`
    +   * will still return `None` for the application.
    +   */
    +  def lease(eventLogSize: Long, isCompressed: Boolean = false): Lease = {
    +    val needed = approximateSize(eventLogSize, isCompressed)
    +    makeRoom(needed)
    +
    +    val perms = PosixFilePermissions.fromString("rwx------")
    +    val tmp = Files.createTempDirectory(tmpStoreDir.toPath(), "appstore",
    +      PosixFilePermissions.asFileAttribute(perms)).toFile()
    +
    +    updateUsage(needed)
    +    new Lease(tmp, needed)
    +  }
    +
    +  /**
    +   * Returns the location of an application store if it's still available. Marks the store as
    +   * being used so that it's not evicted when running out of designated space.
    +   */
    +  def openStore(appId: String, attemptId: Option[String]): Option[File] = {
    +    val storePath = active.synchronized {
    +      val path = appStorePath(appId, attemptId)
    +      if (path.isDirectory()) {
    +        active(appId -> attemptId) = sizeOf(path)
    +        Some(path)
    +      } else {
    +        None
    +      }
    +    }
    +
    +    storePath.foreach { path =>
    +      updateAccessTime(appId, attemptId)
    +    }
    +
    +    storePath
    +  }
    +
    +  /**
    +   * Tell the disk manager that the store for the given application is not being used anymore.
    +   *
    +   * @param delete Whether to delete the store from disk.
    +   */
    +  def release(appId: String, attemptId: Option[String], delete: Boolean = false): Unit = {
    +    // Because LevelDB may modify the structure of the store files even when just reading, update
    +    // the accounting for this application when it's closed.
    +    val oldSizeOpt = active.synchronized {
    +      active.remove(appId -> attemptId)
    +    }
    +
    +    oldSizeOpt.foreach { oldSize =>
    +      val path = appStorePath(appId, attemptId)
    +      updateUsage(-oldSize)
    +      if (path.isDirectory()) {
    +        if (delete) {
    +          FileUtils.deleteDirectory(path)
    +          listing.delete(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +        } else {
    +          updateUsage(sizeOf(path))
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * A non-scientific approximation of how large an app state store will be given the size of the
    +   * event log.
    +   */
    +  def approximateSize(eventLogSize: Long, isCompressed: Boolean): Long = {
    +    val expectedSize = if (isCompressed) {
    +      // For compressed logs, assume that compression reduces the log size a lot, and the disk
    +      // store will actually grow compared to the log size.
    +      eventLogSize * 2
    +    } else {
    +      // For non-compressed logs, assume the disk store will end up at approximately 50% of the
    +      // size of the logs. This is loosely based on empirical evidence.
    +      eventLogSize / 2
    +    }
    +
    +    // Cap the value at 10% of the max size; this assumes that element cleanup will put a cap on
    +    // how large the disk store can get, which may not always be the case.
    +    math.min(expectedSize, maxUsage / 10)
    --- End diff --
    
    Well, all of these heuristics can have a lot of error. Increasing this increases the odds that an existing store will be deleted if another log starts to be parsed in parallel.
    
    Don't really feel strongly either way, so away it goes.


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158513454
  
    --- Diff: core/src/test/scala/org/apache/spark/deploy/history/DiskStoreManagerSuite.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +
    +import org.mockito.Matchers.{any, anyBoolean, anyLong, eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.status.KVUtils
    +import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +class DiskStoreManagerSuite extends SparkFunSuite with BeforeAndAfter {
    +
    +  import config._
    +
    +  private val MAX_USAGE = 3L
    +
    +  private var testDir: File = _
    +  private var store: KVStore = _
    +
    +  before {
    +    testDir = Utils.createTempDir()
    +    store = KVUtils.open(new File(testDir, "listing"), "test")
    +  }
    +
    +  after {
    +    store.close()
    +    if (testDir != null) {
    +      Utils.deleteRecursively(testDir)
    +    }
    +  }
    +
    +  private def mockManager(): DiskStoreManager = {
    +    val conf = new SparkConf().set(MAX_LOCAL_DISK_USAGE, MAX_USAGE)
    +    val manager = spy(new DiskStoreManager(conf, testDir, store, new ManualClock()))
    +    doReturn(0L).when(manager).sizeOf(any(classOf[File]))
    --- End diff --
    
    tests pass without this line ... and its kinda strange, I tried removing it because I was wondering why you would want this


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85320/
    Test PASSed.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85284 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85284/testReport)** for PR 20011 at commit [`2d72dd1`](https://github.com/apache/spark/commit/2d72dd1d08c5fcb174dea14dd0586e1f3f6d4c05).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158515991
  
    --- Diff: core/src/test/scala/org/apache/spark/deploy/history/DiskStoreManagerSuite.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +
    +import org.mockito.Matchers.{any, anyBoolean, anyLong, eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.status.KVUtils
    +import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +class DiskStoreManagerSuite extends SparkFunSuite with BeforeAndAfter {
    +
    +  import config._
    +
    +  private val MAX_USAGE = 3L
    +
    +  private var testDir: File = _
    +  private var store: KVStore = _
    +
    +  before {
    +    testDir = Utils.createTempDir()
    +    store = KVUtils.open(new File(testDir, "listing"), "test")
    +  }
    +
    +  after {
    +    store.close()
    +    if (testDir != null) {
    +      Utils.deleteRecursively(testDir)
    +    }
    +  }
    +
    +  private def mockManager(): DiskStoreManager = {
    +    val conf = new SparkConf().set(MAX_LOCAL_DISK_USAGE, MAX_USAGE)
    +    val manager = spy(new DiskStoreManager(conf, testDir, store, new ManualClock()))
    +    doReturn(0L).when(manager).sizeOf(any(classOf[File]))
    +    doAnswer(new Answer[Long] {
    +      def answer(invocation: InvocationOnMock): Long = {
    +        invocation.getArguments()(0).asInstanceOf[Long]
    +      }
    +    }).when(manager).approximateSize(anyLong(), anyBoolean())
    +    manager
    +  }
    +
    +  private def hasFreeSpace(manager: DiskStoreManager, size: Long): Boolean = {
    +    size <= manager.free()
    +  }
    +
    +  test("leasing space") {
    +    val manager = mockManager()
    +
    +    // Lease all available space.
    +    val lease1 = manager.lease(1)
    +    val lease2 = manager.lease(1)
    +    val lease3 = manager.lease(1)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Revert one lease, get another one.
    +    lease1.rollback()
    +    assert(hasFreeSpace(manager, 1))
    +    assert(!lease1.path.exists())
    +
    +    val lease4 = manager.lease(1)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Committing 2 should bring the "used" space up to 4, so there shouldn't be space left yet.
    +    doReturn(2L).when(manager).sizeOf(meq(lease2.path))
    +    val dst2 = lease2.commit("app2", None)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Rollback 3 and 4, now there should be 1 left.
    --- End diff --
    
    can you switch the naming scheme to be leaseA, leaseB, etc.?  its a bit hard to tell when you're talking about sizes and when you're referring to specific leases.


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158523678
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/DiskStoreManager.scala ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +import java.nio.file.Files
    +import java.nio.file.attribute.PosixFilePermissions
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.{HashMap, ListBuffer}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.status.KVUtils._
    +import org.apache.spark.util.{Clock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +/**
    + * A class used to keep track of disk usage by the SHS, allowing application data to be deleted
    + * from disk when usage exceeds a configurable threshold.
    + *
    + * The goal of the class is not to guarantee that usage will never exceed the threshold; because of
    + * how application data is written, disk usage may temporarily go higher. But, eventually, it
    + * should fall back under the threshold.
    + *
    + * @param conf Spark configuration.
    + * @param path Path where to store application data.
    + * @param listing The listing store, used to persist usage data.
    + * @param clock Clock instance to use.
    + */
    +private class DiskStoreManager(
    +    conf: SparkConf,
    +    path: File,
    +    listing: KVStore,
    +    clock: Clock) extends Logging {
    +
    +  import config._
    +
    +  private val appStoreDir = new File(path, "apps")
    +  if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).")
    +  }
    +
    +  private val tmpStoreDir = new File(path, "temp")
    +  if (!tmpStoreDir.isDirectory() && !tmpStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create temp directory ($tmpStoreDir).")
    +  }
    +
    +  private val maxUsage = conf.get(MAX_LOCAL_DISK_USAGE)
    +  private val currentUsage = new AtomicLong(0L)
    +  private val active = new HashMap[(String, Option[String]), Long]()
    +
    +  def initialize(): Unit = {
    +    updateUsage(sizeOf(appStoreDir))
    +
    +    // Clean up any temporary stores during start up. This assumes that they're leftover from other
    +    // instances and are not useful.
    +    tmpStoreDir.listFiles().foreach(FileUtils.deleteQuietly)
    +
    +    // Go through the recorded store directories and remove any that may have been removed by
    +    // external code.
    +    val orphans = listing.view(classOf[ApplicationStoreInfo]).asScala.filter { info =>
    +      !new File(info.path).exists()
    +    }.toSeq
    +
    +    orphans.foreach { info =>
    +      listing.delete(info.getClass(), info.path)
    +    }
    +  }
    +
    +  /**
    +   * Lease some space from the store. The leased space is calculated as a fraction of the given
    +   * event log size; this is an approximation, and doesn't mean the application store cannot
    +   * outgrow the lease.
    +   *
    +   * If there's not enough space for the lease, other applications might be evicted to make room.
    +   * This method always returns a lease, meaning that it's possible for local disk usage to grow
    +   * past the configured threshold if there aren't enough idle applications to evict.
    +   *
    +   * While the lease is active, the data is written to a temporary location, so `openStore()`
    +   * will still return `None` for the application.
    +   */
    +  def lease(eventLogSize: Long, isCompressed: Boolean = false): Lease = {
    +    val needed = approximateSize(eventLogSize, isCompressed)
    +    makeRoom(needed)
    +
    +    val perms = PosixFilePermissions.fromString("rwx------")
    +    val tmp = Files.createTempDirectory(tmpStoreDir.toPath(), "appstore",
    +      PosixFilePermissions.asFileAttribute(perms)).toFile()
    +
    +    updateUsage(needed)
    +    new Lease(tmp, needed)
    +  }
    +
    +  /**
    +   * Returns the location of an application store if it's still available. Marks the store as
    +   * being used so that it's not evicted when running out of designated space.
    +   */
    +  def openStore(appId: String, attemptId: Option[String]): Option[File] = {
    +    val storePath = active.synchronized {
    +      val path = appStorePath(appId, attemptId)
    +      if (path.isDirectory()) {
    +        active(appId -> attemptId) = sizeOf(path)
    +        Some(path)
    +      } else {
    +        None
    +      }
    +    }
    +
    +    storePath.foreach { path =>
    +      updateAccessTime(appId, attemptId)
    +    }
    +
    +    storePath
    +  }
    +
    +  /**
    +   * Tell the disk manager that the store for the given application is not being used anymore.
    +   *
    +   * @param delete Whether to delete the store from disk.
    +   */
    +  def release(appId: String, attemptId: Option[String], delete: Boolean = false): Unit = {
    +    // Because LevelDB may modify the structure of the store files even when just reading, update
    +    // the accounting for this application when it's closed.
    +    val oldSizeOpt = active.synchronized {
    +      active.remove(appId -> attemptId)
    +    }
    +
    +    oldSizeOpt.foreach { oldSize =>
    +      val path = appStorePath(appId, attemptId)
    +      updateUsage(-oldSize)
    +      if (path.isDirectory()) {
    +        if (delete) {
    +          FileUtils.deleteDirectory(path)
    +          listing.delete(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +        } else {
    +          updateUsage(sizeOf(path))
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * A non-scientific approximation of how large an app state store will be given the size of the
    +   * event log.
    +   */
    +  def approximateSize(eventLogSize: Long, isCompressed: Boolean): Long = {
    +    val expectedSize = if (isCompressed) {
    +      // For compressed logs, assume that compression reduces the log size a lot, and the disk
    +      // store will actually grow compared to the log size.
    +      eventLogSize * 2
    +    } else {
    +      // For non-compressed logs, assume the disk store will end up at approximately 50% of the
    +      // size of the logs. This is loosely based on empirical evidence.
    +      eventLogSize / 2
    +    }
    +
    +    // Cap the value at 10% of the max size; this assumes that element cleanup will put a cap on
    +    // how large the disk store can get, which may not always be the case.
    +    math.min(expectedSize, maxUsage / 10)
    +  }
    +
    +  /** Current free space. Considers space currently leased out too. */
    +  def free(): Long = {
    +    math.max(maxUsage - currentUsage.get(), 0L)
    +  }
    +
    +  private def makeRoom(size: Long): Unit = {
    +    if (free() < size) {
    +      logDebug(s"Not enough free space, looking at candidates for deletion...")
    +      val evicted = new ListBuffer[ApplicationStoreInfo]()
    +      Utils.tryWithResource(
    +        listing.view(classOf[ApplicationStoreInfo]).index("lastAccess").closeableIterator()
    +      ) { iter =>
    +        var needed = size
    +        while (needed > 0 && iter.hasNext()) {
    +          val info = iter.next()
    +          val isActive = active.synchronized {
    +            active.contains(info.appId -> info.attemptId)
    +          }
    +          if (!isActive) {
    +            evicted += info
    +            needed -= info.size
    +          }
    +        }
    +      }
    +
    +      evicted.foreach { info =>
    +        logInfo(s"Deleting store for ${info.appId}/${info.attemptId}.")
    +        FileUtils.deleteDirectory(new File(info.path))
    +        listing.delete(info.getClass(), info.path)
    +      }
    +      logDebug(s"Deleted a total of ${evicted.size} app stores.")
    +    }
    +  }
    +
    +  private def appStorePath(appId: String, attemptId: Option[String]): File = {
    +    val fileName = appId + attemptId.map("_" + _).getOrElse("") + ".ldb"
    +    new File(appStoreDir, fileName)
    +  }
    +
    +  private def updateAccessTime(appId: String, attemptId: Option[String]): Unit = {
    +    val path = appStorePath(appId, attemptId)
    +    val info = ApplicationStoreInfo(path.getAbsolutePath(), clock.getTimeMillis(), appId, attemptId,
    +      sizeOf(path))
    +    listing.write(info)
    +  }
    +
    +  private def updateUsage(delta: Long): Long = {
    +    val updated = currentUsage.addAndGet(delta)
    +    if (updated < 0) {
    +      throw new IllegalStateException(
    +        s"Disk usage tracker went negative (now = $updated, delta = $delta)")
    +    }
    +    updated
    +  }
    +
    +  /** Visible for testing. Return the size of a directory. */
    +  private[history] def sizeOf(path: File): Long = FileUtils.sizeOf(path)
    +
    +  private[history] class Lease(val path: File, private val leased: Long) {
    +
    +    /**
    +     * Commits a lease to its final location, and update accounting information. This method
    +     * marks the application as active, so its store is not available for eviction.
    +     */
    +    def commit(appId: String, attemptId: Option[String]): File = {
    +      val dst = appStorePath(appId, attemptId)
    +
    +      active.synchronized {
    +        require(!active.contains(appId -> attemptId),
    +          s"Cannot commit lease for active application $appId / $attemptId")
    --- End diff --
    
    what is the scenario you're worried about here?  you could have two threads both trying to call commit, and they might both get past this check, and then clobber each other later on, so this is more limited protection.  The idea is just to make sure that you never do a commit while the UI has a store open, as a check of the whole UI cache eviction process?


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158569095
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala ---
    @@ -0,0 +1,310 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +import java.nio.file.Files
    +import java.nio.file.attribute.PosixFilePermissions
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.{HashMap, ListBuffer}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.status.KVUtils._
    +import org.apache.spark.util.{Clock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +/**
    + * A class used to keep track of disk usage by the SHS, allowing application data to be deleted
    + * from disk when usage exceeds a configurable threshold.
    + *
    + * The goal of the class is not to guarantee that usage will never exceed the threshold; because of
    + * how application data is written, disk usage may temporarily go higher. But, eventually, it
    + * should fall back under the threshold.
    + *
    + * @param conf Spark configuration.
    + * @param path Path where to store application data.
    + * @param listing The listing store, used to persist usage data.
    + * @param clock Clock instance to use.
    + */
    +private class HistoryServerDiskManager(
    +    conf: SparkConf,
    +    path: File,
    +    listing: KVStore,
    +    clock: Clock) extends Logging {
    +
    +  import config._
    +
    +  private val appStoreDir = new File(path, "apps")
    +  if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).")
    +  }
    +
    +  private val tmpStoreDir = new File(path, "temp")
    +  if (!tmpStoreDir.isDirectory() && !tmpStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create temp directory ($tmpStoreDir).")
    +  }
    +
    +  private val maxUsage = conf.get(MAX_LOCAL_DISK_USAGE)
    +  private val currentUsage = new AtomicLong(0L)
    +  private val committedUsage = new AtomicLong(0L)
    +  private val active = new HashMap[(String, Option[String]), Long]()
    +
    +  def initialize(): Unit = {
    +    updateUsage(sizeOf(appStoreDir), committed = true)
    +
    +    // Clean up any temporary stores during start up. This assumes that they're leftover from other
    +    // instances and are not useful.
    +    tmpStoreDir.listFiles().foreach(FileUtils.deleteQuietly)
    +
    +    // Go through the recorded store directories and remove any that may have been removed by
    +    // external code.
    +    val orphans = listing.view(classOf[ApplicationStoreInfo]).asScala.filter { info =>
    +      !new File(info.path).exists()
    +    }.toSeq
    +
    +    orphans.foreach { info =>
    +      listing.delete(info.getClass(), info.path)
    +    }
    +  }
    +
    +  /**
    +   * Lease some space from the store. The leased space is calculated as a fraction of the given
    +   * event log size; this is an approximation, and doesn't mean the application store cannot
    +   * outgrow the lease.
    +   *
    +   * If there's not enough space for the lease, other applications might be evicted to make room.
    +   * This method always returns a lease, meaning that it's possible for local disk usage to grow
    +   * past the configured threshold if there aren't enough idle applications to evict.
    +   *
    +   * While the lease is active, the data is written to a temporary location, so `openStore()`
    +   * will still return `None` for the application.
    +   */
    +  def lease(eventLogSize: Long, isCompressed: Boolean = false): Lease = {
    +    val needed = approximateSize(eventLogSize, isCompressed)
    +    makeRoom(needed)
    +
    +    val perms = PosixFilePermissions.fromString("rwx------")
    +    val tmp = Files.createTempDirectory(tmpStoreDir.toPath(), "appstore",
    +      PosixFilePermissions.asFileAttribute(perms)).toFile()
    +
    +    updateUsage(needed)
    +    new Lease(tmp, needed)
    +  }
    +
    +  /**
    +   * Returns the location of an application store if it's still available. Marks the store as
    +   * being used so that it's not evicted when running out of designated space.
    +   */
    +  def openStore(appId: String, attemptId: Option[String]): Option[File] = {
    +    val storePath = active.synchronized {
    +      val path = appStorePath(appId, attemptId)
    +      if (path.isDirectory()) {
    +        active(appId -> attemptId) = sizeOf(path)
    +        Some(path)
    +      } else {
    +        None
    +      }
    +    }
    +
    +    storePath.foreach { path =>
    +      updateAccessTime(appId, attemptId)
    +    }
    +
    +    storePath
    +  }
    +
    +  /**
    +   * Tell the disk manager that the store for the given application is not being used anymore.
    +   *
    +   * @param delete Whether to delete the store from disk.
    +   */
    +  def release(appId: String, attemptId: Option[String], delete: Boolean = false): Unit = {
    +    // Because LevelDB may modify the structure of the store files even when just reading, update
    +    // the accounting for this application when it's closed.
    +    val oldSizeOpt = active.synchronized {
    +      active.remove(appId -> attemptId)
    +    }
    +
    +    oldSizeOpt.foreach { oldSize =>
    +      val path = appStorePath(appId, attemptId)
    +      updateUsage(-oldSize, committed = true)
    +      if (path.isDirectory()) {
    +        if (delete) {
    +          deleteStore(path)
    +        } else {
    +          val newSize = sizeOf(path)
    +          val newInfo = listing.read(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +            .copy(size = newSize)
    +          listing.write(newInfo)
    +          updateUsage(newSize, committed = true)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * A non-scientific approximation of how large an app state store will be given the size of the
    +   * event log.
    +   */
    +  def approximateSize(eventLogSize: Long, isCompressed: Boolean): Long = {
    +    if (isCompressed) {
    +      // For compressed logs, assume that compression reduces the log size a lot, and the disk
    +      // store will actually grow compared to the log size.
    +      eventLogSize * 2
    +    } else {
    +      // For non-compressed logs, assume the disk store will end up at approximately 50% of the
    +      // size of the logs. This is loosely based on empirical evidence.
    +      eventLogSize / 2
    +    }
    +  }
    +
    +  /** Current free space. Considers space currently leased out too. */
    +  def free(): Long = {
    +    math.max(maxUsage - currentUsage.get(), 0L)
    +  }
    +
    +  /** Current committed space. */
    +  def committed(): Long = committedUsage.get()
    +
    +  private def deleteStore(path: File): Unit = {
    +    FileUtils.deleteDirectory(path)
    +    listing.delete(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +  }
    +
    +  private def makeRoom(size: Long): Unit = {
    +    if (free() < size) {
    +      logDebug(s"Not enough free space, looking at candidates for deletion...")
    +      val evicted = new ListBuffer[ApplicationStoreInfo]()
    +      Utils.tryWithResource(
    +        listing.view(classOf[ApplicationStoreInfo]).index("lastAccess").closeableIterator()
    +      ) { iter =>
    +        var needed = size
    +        while (needed > 0 && iter.hasNext()) {
    +          val info = iter.next()
    +          val isActive = active.synchronized {
    +            active.contains(info.appId -> info.attemptId)
    +          }
    +          if (!isActive) {
    +            evicted += info
    +            needed -= info.size
    +          }
    +        }
    +      }
    +
    +      evicted.foreach { info =>
    +        logInfo(s"Deleting store for ${info.appId}/${info.attemptId}.")
    +        deleteStore(new File(info.path))
    +        updateUsage(-info.size, committed = true)
    +      }
    --- End diff --
    
    Yeah, I'll add some more logs.


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158521861
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/DiskStoreManager.scala ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +import java.nio.file.Files
    +import java.nio.file.attribute.PosixFilePermissions
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.{HashMap, ListBuffer}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.status.KVUtils._
    +import org.apache.spark.util.{Clock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +/**
    + * A class used to keep track of disk usage by the SHS, allowing application data to be deleted
    + * from disk when usage exceeds a configurable threshold.
    + *
    + * The goal of the class is not to guarantee that usage will never exceed the threshold; because of
    + * how application data is written, disk usage may temporarily go higher. But, eventually, it
    + * should fall back under the threshold.
    + *
    + * @param conf Spark configuration.
    + * @param path Path where to store application data.
    + * @param listing The listing store, used to persist usage data.
    + * @param clock Clock instance to use.
    + */
    +private class DiskStoreManager(
    +    conf: SparkConf,
    +    path: File,
    +    listing: KVStore,
    +    clock: Clock) extends Logging {
    +
    +  import config._
    +
    +  private val appStoreDir = new File(path, "apps")
    +  if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).")
    +  }
    +
    +  private val tmpStoreDir = new File(path, "temp")
    +  if (!tmpStoreDir.isDirectory() && !tmpStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create temp directory ($tmpStoreDir).")
    +  }
    +
    +  private val maxUsage = conf.get(MAX_LOCAL_DISK_USAGE)
    +  private val currentUsage = new AtomicLong(0L)
    +  private val active = new HashMap[(String, Option[String]), Long]()
    +
    +  def initialize(): Unit = {
    +    updateUsage(sizeOf(appStoreDir))
    +
    +    // Clean up any temporary stores during start up. This assumes that they're leftover from other
    +    // instances and are not useful.
    +    tmpStoreDir.listFiles().foreach(FileUtils.deleteQuietly)
    +
    +    // Go through the recorded store directories and remove any that may have been removed by
    +    // external code.
    +    val orphans = listing.view(classOf[ApplicationStoreInfo]).asScala.filter { info =>
    +      !new File(info.path).exists()
    +    }.toSeq
    +
    +    orphans.foreach { info =>
    +      listing.delete(info.getClass(), info.path)
    +    }
    +  }
    +
    +  /**
    +   * Lease some space from the store. The leased space is calculated as a fraction of the given
    +   * event log size; this is an approximation, and doesn't mean the application store cannot
    +   * outgrow the lease.
    +   *
    +   * If there's not enough space for the lease, other applications might be evicted to make room.
    +   * This method always returns a lease, meaning that it's possible for local disk usage to grow
    +   * past the configured threshold if there aren't enough idle applications to evict.
    +   *
    +   * While the lease is active, the data is written to a temporary location, so `openStore()`
    +   * will still return `None` for the application.
    +   */
    +  def lease(eventLogSize: Long, isCompressed: Boolean = false): Lease = {
    +    val needed = approximateSize(eventLogSize, isCompressed)
    +    makeRoom(needed)
    +
    +    val perms = PosixFilePermissions.fromString("rwx------")
    +    val tmp = Files.createTempDirectory(tmpStoreDir.toPath(), "appstore",
    +      PosixFilePermissions.asFileAttribute(perms)).toFile()
    +
    +    updateUsage(needed)
    +    new Lease(tmp, needed)
    +  }
    +
    +  /**
    +   * Returns the location of an application store if it's still available. Marks the store as
    +   * being used so that it's not evicted when running out of designated space.
    +   */
    +  def openStore(appId: String, attemptId: Option[String]): Option[File] = {
    +    val storePath = active.synchronized {
    +      val path = appStorePath(appId, attemptId)
    +      if (path.isDirectory()) {
    +        active(appId -> attemptId) = sizeOf(path)
    +        Some(path)
    +      } else {
    +        None
    +      }
    +    }
    +
    +    storePath.foreach { path =>
    +      updateAccessTime(appId, attemptId)
    +    }
    +
    +    storePath
    +  }
    +
    +  /**
    +   * Tell the disk manager that the store for the given application is not being used anymore.
    +   *
    +   * @param delete Whether to delete the store from disk.
    +   */
    +  def release(appId: String, attemptId: Option[String], delete: Boolean = false): Unit = {
    +    // Because LevelDB may modify the structure of the store files even when just reading, update
    +    // the accounting for this application when it's closed.
    +    val oldSizeOpt = active.synchronized {
    +      active.remove(appId -> attemptId)
    +    }
    +
    +    oldSizeOpt.foreach { oldSize =>
    +      val path = appStorePath(appId, attemptId)
    +      updateUsage(-oldSize)
    +      if (path.isDirectory()) {
    +        if (delete) {
    +          FileUtils.deleteDirectory(path)
    +          listing.delete(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +        } else {
    +          updateUsage(sizeOf(path))
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * A non-scientific approximation of how large an app state store will be given the size of the
    +   * event log.
    +   */
    +  def approximateSize(eventLogSize: Long, isCompressed: Boolean): Long = {
    +    val expectedSize = if (isCompressed) {
    +      // For compressed logs, assume that compression reduces the log size a lot, and the disk
    +      // store will actually grow compared to the log size.
    +      eventLogSize * 2
    +    } else {
    +      // For non-compressed logs, assume the disk store will end up at approximately 50% of the
    +      // size of the logs. This is loosely based on empirical evidence.
    +      eventLogSize / 2
    +    }
    +
    +    // Cap the value at 10% of the max size; this assumes that element cleanup will put a cap on
    +    // how large the disk store can get, which may not always be the case.
    +    math.min(expectedSize, maxUsage / 10)
    +  }
    +
    +  /** Current free space. Considers space currently leased out too. */
    +  def free(): Long = {
    +    math.max(maxUsage - currentUsage.get(), 0L)
    +  }
    +
    +  private def makeRoom(size: Long): Unit = {
    +    if (free() < size) {
    +      logDebug(s"Not enough free space, looking at candidates for deletion...")
    +      val evicted = new ListBuffer[ApplicationStoreInfo]()
    +      Utils.tryWithResource(
    +        listing.view(classOf[ApplicationStoreInfo]).index("lastAccess").closeableIterator()
    +      ) { iter =>
    +        var needed = size
    +        while (needed > 0 && iter.hasNext()) {
    +          val info = iter.next()
    +          val isActive = active.synchronized {
    +            active.contains(info.appId -> info.attemptId)
    +          }
    +          if (!isActive) {
    +            evicted += info
    +            needed -= info.size
    +          }
    +        }
    +      }
    +
    +      evicted.foreach { info =>
    +        logInfo(s"Deleting store for ${info.appId}/${info.attemptId}.")
    +        FileUtils.deleteDirectory(new File(info.path))
    +        listing.delete(info.getClass(), info.path)
    +      }
    +      logDebug(s"Deleted a total of ${evicted.size} app stores.")
    +    }
    +  }
    +
    +  private def appStorePath(appId: String, attemptId: Option[String]): File = {
    +    val fileName = appId + attemptId.map("_" + _).getOrElse("") + ".ldb"
    +    new File(appStoreDir, fileName)
    +  }
    +
    +  private def updateAccessTime(appId: String, attemptId: Option[String]): Unit = {
    +    val path = appStorePath(appId, attemptId)
    +    val info = ApplicationStoreInfo(path.getAbsolutePath(), clock.getTimeMillis(), appId, attemptId,
    +      sizeOf(path))
    +    listing.write(info)
    +  }
    +
    +  private def updateUsage(delta: Long): Long = {
    +    val updated = currentUsage.addAndGet(delta)
    +    if (updated < 0) {
    +      throw new IllegalStateException(
    +        s"Disk usage tracker went negative (now = $updated, delta = $delta)")
    +    }
    +    updated
    +  }
    +
    +  /** Visible for testing. Return the size of a directory. */
    +  private[history] def sizeOf(path: File): Long = FileUtils.sizeOf(path)
    +
    +  private[history] class Lease(val path: File, private val leased: Long) {
    --- End diff --
    
    can you rename path to tmpPath


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85284/
    Test PASSed.


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158517432
  
    --- Diff: core/src/test/scala/org/apache/spark/deploy/history/DiskStoreManagerSuite.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +
    +import org.mockito.Matchers.{any, anyBoolean, anyLong, eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.status.KVUtils
    +import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +class DiskStoreManagerSuite extends SparkFunSuite with BeforeAndAfter {
    +
    +  import config._
    +
    +  private val MAX_USAGE = 3L
    +
    +  private var testDir: File = _
    +  private var store: KVStore = _
    +
    +  before {
    +    testDir = Utils.createTempDir()
    +    store = KVUtils.open(new File(testDir, "listing"), "test")
    +  }
    +
    +  after {
    +    store.close()
    +    if (testDir != null) {
    +      Utils.deleteRecursively(testDir)
    +    }
    +  }
    +
    +  private def mockManager(): DiskStoreManager = {
    +    val conf = new SparkConf().set(MAX_LOCAL_DISK_USAGE, MAX_USAGE)
    +    val manager = spy(new DiskStoreManager(conf, testDir, store, new ManualClock()))
    +    doReturn(0L).when(manager).sizeOf(any(classOf[File]))
    +    doAnswer(new Answer[Long] {
    +      def answer(invocation: InvocationOnMock): Long = {
    +        invocation.getArguments()(0).asInstanceOf[Long]
    +      }
    +    }).when(manager).approximateSize(anyLong(), anyBoolean())
    +    manager
    +  }
    +
    +  private def hasFreeSpace(manager: DiskStoreManager, size: Long): Boolean = {
    +    size <= manager.free()
    +  }
    +
    +  test("leasing space") {
    +    val manager = mockManager()
    +
    +    // Lease all available space.
    +    val lease1 = manager.lease(1)
    +    val lease2 = manager.lease(1)
    +    val lease3 = manager.lease(1)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Revert one lease, get another one.
    +    lease1.rollback()
    +    assert(hasFreeSpace(manager, 1))
    +    assert(!lease1.path.exists())
    +
    +    val lease4 = manager.lease(1)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Committing 2 should bring the "used" space up to 4, so there shouldn't be space left yet.
    +    doReturn(2L).when(manager).sizeOf(meq(lease2.path))
    +    val dst2 = lease2.commit("app2", None)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Rollback 3 and 4, now there should be 1 left.
    +    lease3.rollback()
    +    lease4.rollback()
    +    assert(hasFreeSpace(manager, 1))
    +    assert(!hasFreeSpace(manager, 2))
    +
    +    // Release app 2 to make it available for eviction.
    +    doReturn(2L).when(manager).sizeOf(meq(dst2))
    +    manager.release("app2", None)
    +
    +    // Lease 1, commit with size 3, replacing previously commited lease 2.
    +    val lease5 = manager.lease(1)
    +    doReturn(3L).when(manager).sizeOf(meq(lease5.path))
    +    lease5.commit("app2", None)
    +    assert(dst2.exists())
    +    assert(!lease5.path.exists())
    +    assert(!hasFreeSpace(manager, 1))
    +    manager.release("app2", None)
    +
    +    // Try a big lease that should cause the committed app to be evicted.
    +    val lease6 = manager.lease(6)
    +    assert(!dst2.exists())
    +    assert(!hasFreeSpace(manager, 1))
    --- End diff --
    
    add a case with getting another lease, even though you're already beyond the size limit (don't think that is covered?)


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85326 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85326/testReport)** for PR 20011 at commit [`931b2d2`](https://github.com/apache/spark/commit/931b2d262aa02880631ca4c693a84fa4c4d12318).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85129 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85129/testReport)** for PR 20011 at commit [`31c2755`](https://github.com/apache/spark/commit/31c2755f5f4f04966c9b0061617563024b6a855f).


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158558556
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/DiskStoreManager.scala ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +import java.nio.file.Files
    +import java.nio.file.attribute.PosixFilePermissions
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.{HashMap, ListBuffer}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.status.KVUtils._
    +import org.apache.spark.util.{Clock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +/**
    + * A class used to keep track of disk usage by the SHS, allowing application data to be deleted
    + * from disk when usage exceeds a configurable threshold.
    + *
    + * The goal of the class is not to guarantee that usage will never exceed the threshold; because of
    + * how application data is written, disk usage may temporarily go higher. But, eventually, it
    + * should fall back under the threshold.
    + *
    + * @param conf Spark configuration.
    + * @param path Path where to store application data.
    + * @param listing The listing store, used to persist usage data.
    + * @param clock Clock instance to use.
    + */
    +private class DiskStoreManager(
    +    conf: SparkConf,
    +    path: File,
    +    listing: KVStore,
    +    clock: Clock) extends Logging {
    +
    +  import config._
    +
    +  private val appStoreDir = new File(path, "apps")
    +  if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).")
    +  }
    +
    +  private val tmpStoreDir = new File(path, "temp")
    +  if (!tmpStoreDir.isDirectory() && !tmpStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create temp directory ($tmpStoreDir).")
    +  }
    +
    +  private val maxUsage = conf.get(MAX_LOCAL_DISK_USAGE)
    +  private val currentUsage = new AtomicLong(0L)
    +  private val active = new HashMap[(String, Option[String]), Long]()
    +
    +  def initialize(): Unit = {
    +    updateUsage(sizeOf(appStoreDir))
    +
    +    // Clean up any temporary stores during start up. This assumes that they're leftover from other
    +    // instances and are not useful.
    +    tmpStoreDir.listFiles().foreach(FileUtils.deleteQuietly)
    +
    +    // Go through the recorded store directories and remove any that may have been removed by
    +    // external code.
    +    val orphans = listing.view(classOf[ApplicationStoreInfo]).asScala.filter { info =>
    +      !new File(info.path).exists()
    +    }.toSeq
    +
    +    orphans.foreach { info =>
    +      listing.delete(info.getClass(), info.path)
    +    }
    +  }
    +
    +  /**
    +   * Lease some space from the store. The leased space is calculated as a fraction of the given
    +   * event log size; this is an approximation, and doesn't mean the application store cannot
    +   * outgrow the lease.
    +   *
    +   * If there's not enough space for the lease, other applications might be evicted to make room.
    +   * This method always returns a lease, meaning that it's possible for local disk usage to grow
    +   * past the configured threshold if there aren't enough idle applications to evict.
    +   *
    +   * While the lease is active, the data is written to a temporary location, so `openStore()`
    +   * will still return `None` for the application.
    +   */
    +  def lease(eventLogSize: Long, isCompressed: Boolean = false): Lease = {
    +    val needed = approximateSize(eventLogSize, isCompressed)
    +    makeRoom(needed)
    +
    +    val perms = PosixFilePermissions.fromString("rwx------")
    +    val tmp = Files.createTempDirectory(tmpStoreDir.toPath(), "appstore",
    +      PosixFilePermissions.asFileAttribute(perms)).toFile()
    +
    +    updateUsage(needed)
    +    new Lease(tmp, needed)
    +  }
    +
    +  /**
    +   * Returns the location of an application store if it's still available. Marks the store as
    +   * being used so that it's not evicted when running out of designated space.
    +   */
    +  def openStore(appId: String, attemptId: Option[String]): Option[File] = {
    +    val storePath = active.synchronized {
    +      val path = appStorePath(appId, attemptId)
    +      if (path.isDirectory()) {
    +        active(appId -> attemptId) = sizeOf(path)
    +        Some(path)
    +      } else {
    +        None
    +      }
    +    }
    +
    +    storePath.foreach { path =>
    +      updateAccessTime(appId, attemptId)
    +    }
    +
    +    storePath
    +  }
    +
    +  /**
    +   * Tell the disk manager that the store for the given application is not being used anymore.
    +   *
    +   * @param delete Whether to delete the store from disk.
    +   */
    +  def release(appId: String, attemptId: Option[String], delete: Boolean = false): Unit = {
    +    // Because LevelDB may modify the structure of the store files even when just reading, update
    +    // the accounting for this application when it's closed.
    +    val oldSizeOpt = active.synchronized {
    +      active.remove(appId -> attemptId)
    +    }
    +
    +    oldSizeOpt.foreach { oldSize =>
    +      val path = appStorePath(appId, attemptId)
    +      updateUsage(-oldSize)
    +      if (path.isDirectory()) {
    +        if (delete) {
    +          FileUtils.deleteDirectory(path)
    +          listing.delete(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +        } else {
    +          updateUsage(sizeOf(path))
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * A non-scientific approximation of how large an app state store will be given the size of the
    +   * event log.
    +   */
    +  def approximateSize(eventLogSize: Long, isCompressed: Boolean): Long = {
    +    val expectedSize = if (isCompressed) {
    +      // For compressed logs, assume that compression reduces the log size a lot, and the disk
    +      // store will actually grow compared to the log size.
    +      eventLogSize * 2
    +    } else {
    +      // For non-compressed logs, assume the disk store will end up at approximately 50% of the
    +      // size of the logs. This is loosely based on empirical evidence.
    +      eventLogSize / 2
    +    }
    +
    +    // Cap the value at 10% of the max size; this assumes that element cleanup will put a cap on
    +    // how large the disk store can get, which may not always be the case.
    +    math.min(expectedSize, maxUsage / 10)
    +  }
    +
    +  /** Current free space. Considers space currently leased out too. */
    +  def free(): Long = {
    +    math.max(maxUsage - currentUsage.get(), 0L)
    +  }
    +
    +  private def makeRoom(size: Long): Unit = {
    +    if (free() < size) {
    +      logDebug(s"Not enough free space, looking at candidates for deletion...")
    +      val evicted = new ListBuffer[ApplicationStoreInfo]()
    +      Utils.tryWithResource(
    +        listing.view(classOf[ApplicationStoreInfo]).index("lastAccess").closeableIterator()
    +      ) { iter =>
    +        var needed = size
    +        while (needed > 0 && iter.hasNext()) {
    +          val info = iter.next()
    +          val isActive = active.synchronized {
    +            active.contains(info.appId -> info.attemptId)
    +          }
    +          if (!isActive) {
    +            evicted += info
    +            needed -= info.size
    +          }
    +        }
    +      }
    +
    +      evicted.foreach { info =>
    +        logInfo(s"Deleting store for ${info.appId}/${info.attemptId}.")
    +        FileUtils.deleteDirectory(new File(info.path))
    +        listing.delete(info.getClass(), info.path)
    +      }
    +      logDebug(s"Deleted a total of ${evicted.size} app stores.")
    +    }
    +  }
    +
    +  private def appStorePath(appId: String, attemptId: Option[String]): File = {
    +    val fileName = appId + attemptId.map("_" + _).getOrElse("") + ".ldb"
    +    new File(appStoreDir, fileName)
    +  }
    +
    +  private def updateAccessTime(appId: String, attemptId: Option[String]): Unit = {
    +    val path = appStorePath(appId, attemptId)
    +    val info = ApplicationStoreInfo(path.getAbsolutePath(), clock.getTimeMillis(), appId, attemptId,
    +      sizeOf(path))
    +    listing.write(info)
    +  }
    +
    +  private def updateUsage(delta: Long): Long = {
    +    val updated = currentUsage.addAndGet(delta)
    +    if (updated < 0) {
    +      throw new IllegalStateException(
    +        s"Disk usage tracker went negative (now = $updated, delta = $delta)")
    +    }
    +    updated
    +  }
    +
    +  /** Visible for testing. Return the size of a directory. */
    +  private[history] def sizeOf(path: File): Long = FileUtils.sizeOf(path)
    +
    +  private[history] class Lease(val path: File, private val leased: Long) {
    +
    +    /**
    +     * Commits a lease to its final location, and update accounting information. This method
    +     * marks the application as active, so its store is not available for eviction.
    +     */
    +    def commit(appId: String, attemptId: Option[String]): File = {
    +      val dst = appStorePath(appId, attemptId)
    +
    +      active.synchronized {
    +        require(!active.contains(appId -> attemptId),
    +          s"Cannot commit lease for active application $appId / $attemptId")
    --- End diff --
    
    This is more of a sanity check that the situation you describe should not happen. The SHS code should be ensuring that there's only a single thread parsing logs for an application, and that while that happens, the app's UI is not loaded. This just asserts that's the case.


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r157641501
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/DiskStoreManager.scala ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +import java.nio.file.Files
    +import java.nio.file.attribute.PosixFilePermissions
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.{HashMap, ListBuffer}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.status.KVUtils._
    +import org.apache.spark.util.{Clock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +/**
    + * A class used to keep track of disk usage by the SHS, allowing application data to be deleted
    + * from disk when usage exceeds a configurable threshold.
    + *
    + * The goal of the class is not to guarantee that usage will never exceed the threshold; because of
    + * how application data is written, disk usage may temporarily go higher. But, eventually, it
    + * should fall back under the threshold.
    + *
    + * @param conf Spark configuration.
    + * @param path Path where to store application data.
    + * @param listing The listing store, used to persist usage data.
    + * @param clock Clock instance to use.
    + */
    +private class DiskStoreManager(
    +    conf: SparkConf,
    +    path: File,
    +    listing: KVStore,
    +    clock: Clock) extends Logging {
    +
    +  import config._
    +
    +  private val appStoreDir = new File(path, "apps")
    +  if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).")
    +  }
    +
    +  private val tmpStoreDir = new File(path, "temp")
    +  if (!tmpStoreDir.isDirectory() && !tmpStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create temp directory ($tmpStoreDir).")
    +  }
    +
    +  private val eventLogSizeRatio = conf.get(EVENT_TO_STORE_SIZE_RATIO)
    +  private val maxUsage = conf.get(MAX_LOCAL_DISK_USAGE)
    +  private val currentUsage = new AtomicLong(0L)
    +  private val active = new HashMap[(String, Option[String]), Long]()
    +
    +  def initialize(): Unit = {
    +    updateUsage(sizeOf(appStoreDir))
    +
    +    // Clean up any temporary stores during start up. This assumes that they're leftover from other
    +    // instances and are not useful.
    +    tmpStoreDir.listFiles().foreach(FileUtils.deleteQuietly)
    +
    +    // Go through the recorded store directories and remove any that may have been removed by
    +    // external code.
    +    val orphans = listing.view(classOf[ApplicationStoreInfo]).asScala.filter { info =>
    +      !new File(info.path).exists()
    +    }.toSeq
    +
    +    orphans.foreach { info =>
    +      listing.delete(info.getClass(), info.path)
    +    }
    +  }
    +
    +  /**
    +   * Lease some space from the store. The leased space is calculated as a fraction of the given
    +   * event log size; this is an approximation, and doesn't mean the application store cannot
    +   * outgrow the lease.
    +   *
    +   * If there's not enough space for the lease, other applications might be evicted to make room.
    +   * This method always returns a lease, meaning that it's possible for local disk usage to grow
    +   * past the configured threshold if there aren't enough idle applications to evict.
    +   *
    +   * While the lease is active, the data is written to a temporary location, so `openStore()`
    +   * will still return `None` for the application.
    +   */
    +  def lease(eventLogSize: Long): Lease = {
    +    val needed = approximateSize(eventLogSize)
    +    makeRoom(needed)
    +
    +    val perms = PosixFilePermissions.fromString("rwx------")
    +    val tmp = Files.createTempDirectory(tmpStoreDir.toPath(), "appstore",
    +      PosixFilePermissions.asFileAttribute(perms)).toFile()
    +
    +    updateUsage(needed)
    +    new Lease(tmp, needed)
    +  }
    +
    +  /**
    +   * Returns whether there's enough free space to create a store for an application event log.
    +   * This uses an approximation of what's the expected size of an application store given the
    +   * size of the event log, since there's no way to really know that relationship up front.
    +   */
    +  def hasFreeSpace(eventLogSize: Long): Boolean = {
    +    approximateSize(eventLogSize) <= free()
    +  }
    +
    +  /**
    +   * Returns the location of an application store if it's still available. Marks the store as
    +   * being used so that it's not evicted when running out of designated space.
    +   */
    +  def openStore(appId: String, attemptId: Option[String]): Option[File] = {
    +    val storePath = active.synchronized {
    +      val path = appStorePath(appId, attemptId)
    +      if (path.isDirectory()) {
    +        active(appId -> attemptId) = sizeOf(path)
    +        Some(path)
    +      } else {
    +        None
    +      }
    +    }
    +
    +    storePath.foreach { path =>
    +      updateAccessTime(appId, attemptId)
    +    }
    +
    +    storePath
    +  }
    +
    +  /**
    +   * Tell the disk manager that the store for the given application is not being used anymore.
    +   *
    +   * @param delete Whether to delete the store from disk.
    +   */
    +  def release(appId: String, attemptId: Option[String], delete: Boolean = false): Unit = {
    +    // Because LevelDB may modify the structure of the store files even when just reading, update
    +    // the accounting for this application when it's closed.
    +    val oldSizeOpt = active.synchronized {
    +      active.remove(appId -> attemptId)
    +    }
    +
    +    oldSizeOpt.foreach { oldSize =>
    +      val path = appStorePath(appId, attemptId)
    +      updateUsage(-oldSize)
    +      if (path.isDirectory()) {
    +        if (delete) {
    +          FileUtils.deleteDirectory(path)
    +          listing.delete(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +        } else {
    +          updateUsage(sizeOf(path))
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * A non-scientific approximation of how large an app state store will be given the size of the
    +   * event log. By default it's 30% of the event log size.
    +   */
    +  private def approximateSize(eventLogSize: Long): Long = {
    +    math.ceil(eventLogSizeRatio * eventLogSize).toLong
    --- End diff --
    
    I'll probably need to look at a better heuristic here. This is probably not a good approximation if the logs are compressed, and #20013 also affects this.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85129/
    Test PASSed.


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/20011


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Some previous comments for this PR can be found at: https://github.com/vanzin/spark/pull/39


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85072 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85072/testReport)** for PR 20011 at commit [`8b43f4d`](https://github.com/apache/spark/commit/8b43f4d2c06ab0a9916113fc216711573db367e7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85326 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85326/testReport)** for PR 20011 at commit [`931b2d2`](https://github.com/apache/spark/commit/931b2d262aa02880631ca4c693a84fa4c4d12318).


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158514239
  
    --- Diff: core/src/test/scala/org/apache/spark/deploy/history/DiskStoreManagerSuite.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +
    +import org.mockito.Matchers.{any, anyBoolean, anyLong, eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.status.KVUtils
    +import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +class DiskStoreManagerSuite extends SparkFunSuite with BeforeAndAfter {
    +
    +  import config._
    +
    +  private val MAX_USAGE = 3L
    +
    +  private var testDir: File = _
    +  private var store: KVStore = _
    +
    +  before {
    +    testDir = Utils.createTempDir()
    +    store = KVUtils.open(new File(testDir, "listing"), "test")
    +  }
    +
    +  after {
    +    store.close()
    +    if (testDir != null) {
    +      Utils.deleteRecursively(testDir)
    +    }
    +  }
    +
    +  private def mockManager(): DiskStoreManager = {
    +    val conf = new SparkConf().set(MAX_LOCAL_DISK_USAGE, MAX_USAGE)
    +    val manager = spy(new DiskStoreManager(conf, testDir, store, new ManualClock()))
    +    doReturn(0L).when(manager).sizeOf(any(classOf[File]))
    +    doAnswer(new Answer[Long] {
    +      def answer(invocation: InvocationOnMock): Long = {
    +        invocation.getArguments()(0).asInstanceOf[Long]
    +      }
    +    }).when(manager).approximateSize(anyLong(), anyBoolean())
    +    manager
    +  }
    +
    +  private def hasFreeSpace(manager: DiskStoreManager, size: Long): Boolean = {
    +    size <= manager.free()
    +  }
    +
    +  test("leasing space") {
    +    val manager = mockManager()
    +
    +    // Lease all available space.
    +    val lease1 = manager.lease(1)
    +    val lease2 = manager.lease(1)
    +    val lease3 = manager.lease(1)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Revert one lease, get another one.
    +    lease1.rollback()
    +    assert(hasFreeSpace(manager, 1))
    +    assert(!lease1.path.exists())
    +
    +    val lease4 = manager.lease(1)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Committing 2 should bring the "used" space up to 4, so there shouldn't be space left yet.
    +    doReturn(2L).when(manager).sizeOf(meq(lease2.path))
    +    val dst2 = lease2.commit("app2", None)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Rollback 3 and 4, now there should be 1 left.
    +    lease3.rollback()
    +    lease4.rollback()
    +    assert(hasFreeSpace(manager, 1))
    +    assert(!hasFreeSpace(manager, 2))
    --- End diff --
    
    nit: I don't think `hasFreeSpace` actually helps at all,  here you could replace both of these with `assert(manager.free() === 1)` and everywhere else do `assert(manager.free() === 0)` etc.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85129 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85129/testReport)** for PR 20011 at commit [`31c2755`](https://github.com/apache/spark/commit/31c2755f5f4f04966c9b0061617563024b6a855f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158390726
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/DiskStoreManager.scala ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +import java.nio.file.Files
    +import java.nio.file.attribute.PosixFilePermissions
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.{HashMap, ListBuffer}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.status.KVUtils._
    +import org.apache.spark.util.{Clock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +/**
    + * A class used to keep track of disk usage by the SHS, allowing application data to be deleted
    + * from disk when usage exceeds a configurable threshold.
    + *
    + * The goal of the class is not to guarantee that usage will never exceed the threshold; because of
    + * how application data is written, disk usage may temporarily go higher. But, eventually, it
    + * should fall back under the threshold.
    + *
    + * @param conf Spark configuration.
    + * @param path Path where to store application data.
    + * @param listing The listing store, used to persist usage data.
    + * @param clock Clock instance to use.
    + */
    +private class DiskStoreManager(
    +    conf: SparkConf,
    +    path: File,
    +    listing: KVStore,
    +    clock: Clock) extends Logging {
    +
    +  import config._
    +
    +  private val appStoreDir = new File(path, "apps")
    +  if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).")
    +  }
    +
    +  private val tmpStoreDir = new File(path, "temp")
    +  if (!tmpStoreDir.isDirectory() && !tmpStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create temp directory ($tmpStoreDir).")
    +  }
    +
    +  private val maxUsage = conf.get(MAX_LOCAL_DISK_USAGE)
    +  private val currentUsage = new AtomicLong(0L)
    +  private val active = new HashMap[(String, Option[String]), Long]()
    +
    +  def initialize(): Unit = {
    +    updateUsage(sizeOf(appStoreDir))
    +
    +    // Clean up any temporary stores during start up. This assumes that they're leftover from other
    +    // instances and are not useful.
    +    tmpStoreDir.listFiles().foreach(FileUtils.deleteQuietly)
    +
    +    // Go through the recorded store directories and remove any that may have been removed by
    +    // external code.
    +    val orphans = listing.view(classOf[ApplicationStoreInfo]).asScala.filter { info =>
    +      !new File(info.path).exists()
    +    }.toSeq
    +
    +    orphans.foreach { info =>
    +      listing.delete(info.getClass(), info.path)
    +    }
    +  }
    +
    +  /**
    +   * Lease some space from the store. The leased space is calculated as a fraction of the given
    +   * event log size; this is an approximation, and doesn't mean the application store cannot
    +   * outgrow the lease.
    +   *
    +   * If there's not enough space for the lease, other applications might be evicted to make room.
    +   * This method always returns a lease, meaning that it's possible for local disk usage to grow
    +   * past the configured threshold if there aren't enough idle applications to evict.
    +   *
    +   * While the lease is active, the data is written to a temporary location, so `openStore()`
    +   * will still return `None` for the application.
    +   */
    +  def lease(eventLogSize: Long, isCompressed: Boolean = false): Lease = {
    +    val needed = approximateSize(eventLogSize, isCompressed)
    +    makeRoom(needed)
    +
    +    val perms = PosixFilePermissions.fromString("rwx------")
    +    val tmp = Files.createTempDirectory(tmpStoreDir.toPath(), "appstore",
    +      PosixFilePermissions.asFileAttribute(perms)).toFile()
    +
    +    updateUsage(needed)
    +    new Lease(tmp, needed)
    +  }
    +
    +  /**
    +   * Returns the location of an application store if it's still available. Marks the store as
    +   * being used so that it's not evicted when running out of designated space.
    +   */
    +  def openStore(appId: String, attemptId: Option[String]): Option[File] = {
    +    val storePath = active.synchronized {
    +      val path = appStorePath(appId, attemptId)
    +      if (path.isDirectory()) {
    +        active(appId -> attemptId) = sizeOf(path)
    +        Some(path)
    +      } else {
    +        None
    +      }
    +    }
    +
    +    storePath.foreach { path =>
    +      updateAccessTime(appId, attemptId)
    +    }
    +
    +    storePath
    +  }
    +
    +  /**
    +   * Tell the disk manager that the store for the given application is not being used anymore.
    +   *
    +   * @param delete Whether to delete the store from disk.
    +   */
    +  def release(appId: String, attemptId: Option[String], delete: Boolean = false): Unit = {
    +    // Because LevelDB may modify the structure of the store files even when just reading, update
    +    // the accounting for this application when it's closed.
    +    val oldSizeOpt = active.synchronized {
    +      active.remove(appId -> attemptId)
    +    }
    +
    +    oldSizeOpt.foreach { oldSize =>
    +      val path = appStorePath(appId, attemptId)
    +      updateUsage(-oldSize)
    +      if (path.isDirectory()) {
    +        if (delete) {
    +          FileUtils.deleteDirectory(path)
    +          listing.delete(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +        } else {
    +          updateUsage(sizeOf(path))
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * A non-scientific approximation of how large an app state store will be given the size of the
    +   * event log.
    +   */
    +  def approximateSize(eventLogSize: Long, isCompressed: Boolean): Long = {
    +    val expectedSize = if (isCompressed) {
    +      // For compressed logs, assume that compression reduces the log size a lot, and the disk
    +      // store will actually grow compared to the log size.
    +      eventLogSize * 2
    +    } else {
    +      // For non-compressed logs, assume the disk store will end up at approximately 50% of the
    +      // size of the logs. This is loosely based on empirical evidence.
    +      eventLogSize / 2
    +    }
    +
    +    // Cap the value at 10% of the max size; this assumes that element cleanup will put a cap on
    +    // how large the disk store can get, which may not always be the case.
    +    math.min(expectedSize, maxUsage / 10)
    --- End diff --
    
    this seems really arbitrary, and could have a ton of error.  Do we really need it at all?


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158558838
  
    --- Diff: core/src/test/scala/org/apache/spark/deploy/history/DiskStoreManagerSuite.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +
    +import org.mockito.Matchers.{any, anyBoolean, anyLong, eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.status.KVUtils
    +import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +class DiskStoreManagerSuite extends SparkFunSuite with BeforeAndAfter {
    +
    +  import config._
    +
    +  private val MAX_USAGE = 3L
    +
    +  private var testDir: File = _
    +  private var store: KVStore = _
    +
    +  before {
    +    testDir = Utils.createTempDir()
    +    store = KVUtils.open(new File(testDir, "listing"), "test")
    +  }
    +
    +  after {
    +    store.close()
    +    if (testDir != null) {
    +      Utils.deleteRecursively(testDir)
    +    }
    +  }
    +
    +  private def mockManager(): DiskStoreManager = {
    +    val conf = new SparkConf().set(MAX_LOCAL_DISK_USAGE, MAX_USAGE)
    +    val manager = spy(new DiskStoreManager(conf, testDir, store, new ManualClock()))
    +    doReturn(0L).when(manager).sizeOf(any(classOf[File]))
    --- End diff --
    
    Just added in case the method was called with an unexpected file. Will remove.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85284 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85284/testReport)** for PR 20011 at commit [`2d72dd1`](https://github.com/apache/spark/commit/2d72dd1d08c5fcb174dea14dd0586e1f3f6d4c05).


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    merged to master


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158510545
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/DiskStoreManager.scala ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +import java.nio.file.Files
    +import java.nio.file.attribute.PosixFilePermissions
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.{HashMap, ListBuffer}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.status.KVUtils._
    +import org.apache.spark.util.{Clock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +/**
    + * A class used to keep track of disk usage by the SHS, allowing application data to be deleted
    + * from disk when usage exceeds a configurable threshold.
    + *
    + * The goal of the class is not to guarantee that usage will never exceed the threshold; because of
    + * how application data is written, disk usage may temporarily go higher. But, eventually, it
    + * should fall back under the threshold.
    + *
    + * @param conf Spark configuration.
    + * @param path Path where to store application data.
    + * @param listing The listing store, used to persist usage data.
    + * @param clock Clock instance to use.
    + */
    +private class DiskStoreManager(
    --- End diff --
    
    there is another very different `DiskStore`, can we name this something else so its easier to tell they're unrelated?  `KVStoreDiskManager`?


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158568388
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala ---
    @@ -0,0 +1,310 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +import java.nio.file.Files
    +import java.nio.file.attribute.PosixFilePermissions
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.{HashMap, ListBuffer}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.status.KVUtils._
    +import org.apache.spark.util.{Clock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +/**
    + * A class used to keep track of disk usage by the SHS, allowing application data to be deleted
    + * from disk when usage exceeds a configurable threshold.
    + *
    + * The goal of the class is not to guarantee that usage will never exceed the threshold; because of
    + * how application data is written, disk usage may temporarily go higher. But, eventually, it
    + * should fall back under the threshold.
    + *
    + * @param conf Spark configuration.
    + * @param path Path where to store application data.
    + * @param listing The listing store, used to persist usage data.
    + * @param clock Clock instance to use.
    + */
    +private class HistoryServerDiskManager(
    +    conf: SparkConf,
    +    path: File,
    +    listing: KVStore,
    +    clock: Clock) extends Logging {
    +
    +  import config._
    +
    +  private val appStoreDir = new File(path, "apps")
    +  if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).")
    +  }
    +
    +  private val tmpStoreDir = new File(path, "temp")
    +  if (!tmpStoreDir.isDirectory() && !tmpStoreDir.mkdir()) {
    +    throw new IllegalArgumentException(s"Failed to create temp directory ($tmpStoreDir).")
    +  }
    +
    +  private val maxUsage = conf.get(MAX_LOCAL_DISK_USAGE)
    +  private val currentUsage = new AtomicLong(0L)
    +  private val committedUsage = new AtomicLong(0L)
    +  private val active = new HashMap[(String, Option[String]), Long]()
    +
    +  def initialize(): Unit = {
    +    updateUsage(sizeOf(appStoreDir), committed = true)
    +
    +    // Clean up any temporary stores during start up. This assumes that they're leftover from other
    +    // instances and are not useful.
    +    tmpStoreDir.listFiles().foreach(FileUtils.deleteQuietly)
    +
    +    // Go through the recorded store directories and remove any that may have been removed by
    +    // external code.
    +    val orphans = listing.view(classOf[ApplicationStoreInfo]).asScala.filter { info =>
    +      !new File(info.path).exists()
    +    }.toSeq
    +
    +    orphans.foreach { info =>
    +      listing.delete(info.getClass(), info.path)
    +    }
    +  }
    +
    +  /**
    +   * Lease some space from the store. The leased space is calculated as a fraction of the given
    +   * event log size; this is an approximation, and doesn't mean the application store cannot
    +   * outgrow the lease.
    +   *
    +   * If there's not enough space for the lease, other applications might be evicted to make room.
    +   * This method always returns a lease, meaning that it's possible for local disk usage to grow
    +   * past the configured threshold if there aren't enough idle applications to evict.
    +   *
    +   * While the lease is active, the data is written to a temporary location, so `openStore()`
    +   * will still return `None` for the application.
    +   */
    +  def lease(eventLogSize: Long, isCompressed: Boolean = false): Lease = {
    +    val needed = approximateSize(eventLogSize, isCompressed)
    +    makeRoom(needed)
    +
    +    val perms = PosixFilePermissions.fromString("rwx------")
    +    val tmp = Files.createTempDirectory(tmpStoreDir.toPath(), "appstore",
    +      PosixFilePermissions.asFileAttribute(perms)).toFile()
    +
    +    updateUsage(needed)
    +    new Lease(tmp, needed)
    +  }
    +
    +  /**
    +   * Returns the location of an application store if it's still available. Marks the store as
    +   * being used so that it's not evicted when running out of designated space.
    +   */
    +  def openStore(appId: String, attemptId: Option[String]): Option[File] = {
    +    val storePath = active.synchronized {
    +      val path = appStorePath(appId, attemptId)
    +      if (path.isDirectory()) {
    +        active(appId -> attemptId) = sizeOf(path)
    +        Some(path)
    +      } else {
    +        None
    +      }
    +    }
    +
    +    storePath.foreach { path =>
    +      updateAccessTime(appId, attemptId)
    +    }
    +
    +    storePath
    +  }
    +
    +  /**
    +   * Tell the disk manager that the store for the given application is not being used anymore.
    +   *
    +   * @param delete Whether to delete the store from disk.
    +   */
    +  def release(appId: String, attemptId: Option[String], delete: Boolean = false): Unit = {
    +    // Because LevelDB may modify the structure of the store files even when just reading, update
    +    // the accounting for this application when it's closed.
    +    val oldSizeOpt = active.synchronized {
    +      active.remove(appId -> attemptId)
    +    }
    +
    +    oldSizeOpt.foreach { oldSize =>
    +      val path = appStorePath(appId, attemptId)
    +      updateUsage(-oldSize, committed = true)
    +      if (path.isDirectory()) {
    +        if (delete) {
    +          deleteStore(path)
    +        } else {
    +          val newSize = sizeOf(path)
    +          val newInfo = listing.read(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +            .copy(size = newSize)
    +          listing.write(newInfo)
    +          updateUsage(newSize, committed = true)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * A non-scientific approximation of how large an app state store will be given the size of the
    +   * event log.
    +   */
    +  def approximateSize(eventLogSize: Long, isCompressed: Boolean): Long = {
    +    if (isCompressed) {
    +      // For compressed logs, assume that compression reduces the log size a lot, and the disk
    +      // store will actually grow compared to the log size.
    +      eventLogSize * 2
    +    } else {
    +      // For non-compressed logs, assume the disk store will end up at approximately 50% of the
    +      // size of the logs. This is loosely based on empirical evidence.
    +      eventLogSize / 2
    +    }
    +  }
    +
    +  /** Current free space. Considers space currently leased out too. */
    +  def free(): Long = {
    +    math.max(maxUsage - currentUsage.get(), 0L)
    +  }
    +
    +  /** Current committed space. */
    +  def committed(): Long = committedUsage.get()
    +
    +  private def deleteStore(path: File): Unit = {
    +    FileUtils.deleteDirectory(path)
    +    listing.delete(classOf[ApplicationStoreInfo], path.getAbsolutePath())
    +  }
    +
    +  private def makeRoom(size: Long): Unit = {
    +    if (free() < size) {
    +      logDebug(s"Not enough free space, looking at candidates for deletion...")
    +      val evicted = new ListBuffer[ApplicationStoreInfo]()
    +      Utils.tryWithResource(
    +        listing.view(classOf[ApplicationStoreInfo]).index("lastAccess").closeableIterator()
    +      ) { iter =>
    +        var needed = size
    +        while (needed > 0 && iter.hasNext()) {
    +          val info = iter.next()
    +          val isActive = active.synchronized {
    +            active.contains(info.appId -> info.attemptId)
    +          }
    +          if (!isActive) {
    +            evicted += info
    +            needed -= info.size
    +          }
    +        }
    +      }
    +
    +      evicted.foreach { info =>
    +        logInfo(s"Deleting store for ${info.appId}/${info.attemptId}.")
    +        deleteStore(new File(info.path))
    +        updateUsage(-info.size, committed = true)
    +      }
    --- End diff --
    
    I like the change to track committed space.  I'm wondering if we should log more here, even at Info level, summarizing the available space (I can see users complaining that they configured it to 10GB but its taking 12GB and wanting an explanation -- would be nice if the logs already had all the relevant info).  Maybe something like
    
    ```
    val totalNeeded = size
    var activeSize = 0
    ...
    if (!isActive) {
      evicted += info
      needed -= info.size
    } else {
      activeSize += info.size
    }
    ...
    logInfo(s"Deleted ${evicted.size} apps to free ${Utils.bytesToString(needed)})
    if (needed > 0) {
      val current = currentUsage.get()
      val leased = Utils.bytesToString(current - committedUsage.get())
      logInfo(s"Did not free requested ${Utils.bytesToString(totalNeeded)}.  Current usage is ${Utils.bytesToString(current).  $leased (estimated) used by apps actively updating their kvstores; ${Utils.bytesToString(active)} used by active applications.")
    }


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85320 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85320/testReport)** for PR 20011 at commit [`be3f130`](https://github.com/apache/spark/commit/be3f1307f0edffdc7c9457ec960781cd28b07bf8).


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85072/
    Test PASSed.


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85320 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85320/testReport)** for PR 20011 at commit [`be3f130`](https://github.com/apache/spark/commit/be3f1307f0edffdc7c9457ec960781cd28b07bf8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20011: [SPARK-20654][core] Add config to limit disk usag...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20011#discussion_r158516782
  
    --- Diff: core/src/test/scala/org/apache/spark/deploy/history/DiskStoreManagerSuite.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.deploy.history
    +
    +import java.io.File
    +
    +import org.mockito.Matchers.{any, anyBoolean, anyLong, eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.status.KVUtils
    +import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.util.kvstore.KVStore
    +
    +class DiskStoreManagerSuite extends SparkFunSuite with BeforeAndAfter {
    +
    +  import config._
    +
    +  private val MAX_USAGE = 3L
    +
    +  private var testDir: File = _
    +  private var store: KVStore = _
    +
    +  before {
    +    testDir = Utils.createTempDir()
    +    store = KVUtils.open(new File(testDir, "listing"), "test")
    +  }
    +
    +  after {
    +    store.close()
    +    if (testDir != null) {
    +      Utils.deleteRecursively(testDir)
    +    }
    +  }
    +
    +  private def mockManager(): DiskStoreManager = {
    +    val conf = new SparkConf().set(MAX_LOCAL_DISK_USAGE, MAX_USAGE)
    +    val manager = spy(new DiskStoreManager(conf, testDir, store, new ManualClock()))
    +    doReturn(0L).when(manager).sizeOf(any(classOf[File]))
    +    doAnswer(new Answer[Long] {
    +      def answer(invocation: InvocationOnMock): Long = {
    +        invocation.getArguments()(0).asInstanceOf[Long]
    +      }
    +    }).when(manager).approximateSize(anyLong(), anyBoolean())
    +    manager
    +  }
    +
    +  private def hasFreeSpace(manager: DiskStoreManager, size: Long): Boolean = {
    +    size <= manager.free()
    +  }
    +
    +  test("leasing space") {
    +    val manager = mockManager()
    +
    +    // Lease all available space.
    +    val lease1 = manager.lease(1)
    +    val lease2 = manager.lease(1)
    +    val lease3 = manager.lease(1)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Revert one lease, get another one.
    +    lease1.rollback()
    +    assert(hasFreeSpace(manager, 1))
    +    assert(!lease1.path.exists())
    +
    +    val lease4 = manager.lease(1)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Committing 2 should bring the "used" space up to 4, so there shouldn't be space left yet.
    +    doReturn(2L).when(manager).sizeOf(meq(lease2.path))
    +    val dst2 = lease2.commit("app2", None)
    +    assert(!hasFreeSpace(manager, 1))
    +
    +    // Rollback 3 and 4, now there should be 1 left.
    +    lease3.rollback()
    +    lease4.rollback()
    +    assert(hasFreeSpace(manager, 1))
    +    assert(!hasFreeSpace(manager, 2))
    +
    +    // Release app 2 to make it available for eviction.
    +    doReturn(2L).when(manager).sizeOf(meq(dst2))
    +    manager.release("app2", None)
    +
    +    // Lease 1, commit with size 3, replacing previously commited lease 2.
    +    val lease5 = manager.lease(1)
    +    doReturn(3L).when(manager).sizeOf(meq(lease5.path))
    +    lease5.commit("app2", None)
    +    assert(dst2.exists())
    --- End diff --
    
    would help here to have a comment explaining the case this really represents -- replaying an app because the event log has been updated, so replacing the old kvstore.  Also just to make that clear you could add
    
    ```
    val dst5 = lease5.commit("app2", None)
    assert(dst5 === dst2)
    ```
    (at first I thought "app2" was a copy/paste mistake here)


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    **[Test build #85072 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85072/testReport)** for PR 20011 at commit [`8b43f4d`](https://github.com/apache/spark/commit/8b43f4d2c06ab0a9916113fc216711573db367e7).


---

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


[GitHub] spark issue #20011: [SPARK-20654][core] Add config to limit disk usage of th...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20011
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85326/
    Test PASSed.


---

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