|
| 1 | +package io.delta.storage.integration |
| 2 | + |
| 3 | +import io.delta.storage.internal.{FileNameUtils, S3LogStoreUtil} |
| 4 | +import org.apache.hadoop.conf.Configuration |
| 5 | +import org.apache.hadoop.fs.Path |
| 6 | +import org.apache.hadoop.fs.s3a.S3AFileSystem |
| 7 | +import org.scalatest.Tag |
| 8 | +import org.scalatest.funsuite.AnyFunSuite |
| 9 | + |
| 10 | +import java.net.URI |
| 11 | +import scala.math.max |
| 12 | +import scala.math.ceil |
| 13 | + |
| 14 | +/** |
| 15 | + * These integration tests are executed by setting the |
| 16 | + * environment variables |
| 17 | + * S3_LOG_STORE_UTIL_TEST_BUCKET=some-s3-bucket-name |
| 18 | + * S3_LOG_STORE_UTIL_TEST_RUN_UID=some-uuid-for-test-run |
| 19 | + * and running |
| 20 | + * python run-integration-tests.py --s3-log-store-util-only |
| 21 | + * |
| 22 | + * Alternatively you can set the environment variables |
| 23 | + * S3_LOG_STORE_UTIL_TEST_ENABLED=true |
| 24 | + * S3_LOG_STORE_UTIL_TEST_BUCKET=some-s3-bucket-name |
| 25 | + * S3_LOG_STORE_UTIL_TEST_RUN_UID=some-uuid-for-test-run |
| 26 | + * and run the tests in this suite using your preferred |
| 27 | + * test execution mechanism (e.g., the IDE or sbt) |
| 28 | + * |
| 29 | + * S3_LOG_STORE_UTIL_TEST_BUCKET is the name of the S3 bucket used for the test. |
| 30 | + * S3_LOG_STORE_UTIL_TEST_RUN_UID is a prefix for all keys used in the test. |
| 31 | + * This is useful for isolating multiple test runs. |
| 32 | + */ |
| 33 | +class S3LogStoreUtilIntegrationTest extends AnyFunSuite { |
| 34 | + private val runIntegrationTests: Boolean = |
| 35 | + Option(System.getenv("S3_LOG_STORE_UTIL_TEST_ENABLED")).exists(_.toBoolean) |
| 36 | + private val bucket = System.getenv("S3_LOG_STORE_UTIL_TEST_BUCKET") |
| 37 | + private val testRunUID = |
| 38 | + System.getenv("S3_LOG_STORE_UTIL_TEST_RUN_UID") // Prefix for all S3 keys in the current run |
| 39 | + private lazy val fs: S3AFileSystem = { |
| 40 | + val fs = new S3AFileSystem() |
| 41 | + fs.initialize(new URI(s"s3a://$bucket"), configuration) |
| 42 | + fs |
| 43 | + } |
| 44 | + private val maxKeys = 2 |
| 45 | + private val configuration = new Configuration() |
| 46 | + configuration.set( // for local testing only |
| 47 | + "fs.s3a.aws.credentials.provider", |
| 48 | + "com.amazonaws.auth.profile.ProfileCredentialsProvider", |
| 49 | + ) |
| 50 | + configuration.set("fs.s3a.paging.maximum", maxKeys.toString) |
| 51 | + |
| 52 | + private def touch(key: String) { |
| 53 | + fs.create(new Path(s"s3a://$bucket/$key")).close() |
| 54 | + } |
| 55 | + |
| 56 | + private def key(table: String, version: Int): String = |
| 57 | + s"$testRunUID/$table/_delta_log/%020d.json".format(version) |
| 58 | + |
| 59 | + private def path(table: String, version: Int): Path = |
| 60 | + new Path(s"s3a://$bucket/${key(table, version)}") |
| 61 | + |
| 62 | + private def version(path: Path): Long = FileNameUtils.deltaVersion(path) |
| 63 | + |
| 64 | + private val integrationTestTag = Tag("IntegrationTest") |
| 65 | + |
| 66 | + def integrationTest(name: String)(testFun: => Any): Unit = |
| 67 | + if (runIntegrationTests) test(name, integrationTestTag)(testFun) |
| 68 | + |
| 69 | + def testCase(testName: String, numKeys: Int): Unit = integrationTest(testName) { |
| 70 | + // Setup delta log |
| 71 | + (1 to numKeys).foreach(v => touch(s"$testRunUID/$testName/_delta_log/%020d.json".format(v))) |
| 72 | + |
| 73 | + // Check number of S3 requests and correct listing |
| 74 | + (1 to numKeys + 2).foreach(v => { |
| 75 | + val startCount = fs.getIOStatistics.counters().get("object_list_request") + |
| 76 | + fs.getIOStatistics.counters().get("object_continue_list_request") |
| 77 | + val resolvedPath = path(testName, v) |
| 78 | + val response = S3LogStoreUtil.s3ListFromArray(fs, resolvedPath, resolvedPath.getParent) |
| 79 | + val endCount = fs.getIOStatistics.counters().get("object_list_request") + |
| 80 | + fs.getIOStatistics.counters().get("object_continue_list_request") |
| 81 | + // Check that we don't do more S3 list requests than necessary |
| 82 | + val numberOfKeysToList = numKeys - (v - 1) |
| 83 | + val optimalNumberOfListRequests = |
| 84 | + max(ceil(numberOfKeysToList / maxKeys.toDouble).toInt, 1) |
| 85 | + val actualNumberOfListRequests = endCount - startCount |
| 86 | + assert(optimalNumberOfListRequests == actualNumberOfListRequests) |
| 87 | + // Check that we get consecutive versions from v to the max version. The smallest version is 1 |
| 88 | + assert((max(1, v) to numKeys) == response.map(r => version(r.getPath)).toSeq) |
| 89 | + }) |
| 90 | + } |
| 91 | + |
| 92 | + integrationTest("setup empty delta log") { |
| 93 | + touch(s"$testRunUID/empty/some.json") |
| 94 | + } |
| 95 | + |
| 96 | + testCase("empty", 0) |
| 97 | + |
| 98 | + testCase("small", 1) |
| 99 | + |
| 100 | + testCase("medium", maxKeys) |
| 101 | + |
| 102 | + testCase("large", 10 * maxKeys) |
| 103 | + |
| 104 | +} |
0 commit comments