Skip to content
Closed
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
added test
  • Loading branch information
brkyvz committed Nov 8, 2016
commit e0c164b86ce7cff0750615c16b2ee41905bfb875
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,14 @@

package org.apache.spark.sql.execution.streaming.state

import java.io.File
import java.io.{File, IOException}
import java.net.URI

import scala.collection.mutable
import scala.util.Random

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem}
import org.scalatest.{BeforeAndAfter, PrivateMethodTester}
import org.scalatest.concurrent.Eventually._
import org.scalatest.time.SpanSugar._
Expand Down Expand Up @@ -455,6 +456,18 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth
}
}

test("commit fails when rename fails") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you add the jira number here.

import RenameReturnsFalseFileSystem._
val dir = scheme + "://" + Utils.createDirectory(tempDir, Random.nextString(5)).toString
val conf = new Configuration()
conf.set(s"fs.$scheme.impl", classOf[RenameReturnsFalseFileSystem].getName)
val provider = newStoreProvider(dir = dir, hadoopConf = conf)
val store = provider.getStore(0)
put(store, "a", 0)
val e = intercept[IllegalStateException](store.commit())
assert(e.getCause.getMessage.contains("Failed to rename"))
}

def getDataFromFiles(
provider: HDFSBackedStateStoreProvider,
version: Int = -1): Set[(String, Int)] = {
Expand Down Expand Up @@ -524,17 +537,18 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth
def newStoreProvider(
opId: Long = Random.nextLong,
partition: Int = 0,
minDeltasForSnapshot: Int = SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.defaultValue.get
minDeltasForSnapshot: Int = SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.defaultValue.get,
dir: String = Utils.createDirectory(tempDir, Random.nextString(5)).toString,
hadoopConf: Configuration = new Configuration()
): HDFSBackedStateStoreProvider = {
val dir = Utils.createDirectory(tempDir, Random.nextString(5)).toString
val sqlConf = new SQLConf()
sqlConf.setConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT, minDeltasForSnapshot)
new HDFSBackedStateStoreProvider(
StateStoreId(dir, opId, partition),
keySchema,
valueSchema,
new StateStoreConf(sqlConf),
new Configuration())
hadoopConf)
}

def remove(store: StateStore, condition: String => Boolean): Unit = {
Expand Down Expand Up @@ -598,3 +612,19 @@ private[state] object StateStoreSuite {
}}.toSet
}
}

/** Fake FileSystem to test whether the method `fs.exists` is called during
* `DataSource.resolveRelation`.
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wrong docs.

class RenameReturnsFalseFileSystem extends RawLocalFileSystem {
import RenameReturnsFalseFileSystem._
override def getUri: URI = {
URI.create(s"$scheme:///")
}

override def rename(src: Path, dst: Path): Boolean = false
}

object RenameReturnsFalseFileSystem {
val scheme = s"StateStoreSuite${math.abs(Random.nextInt)}fs"
}