Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Refresh presigned urls for cdf queries in v0.5 #226

Merged
merged 1 commit into from
Dec 13, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,11 @@

package io.delta.sharing.spark

import java.lang.ref.WeakReference

import scala.collection.mutable.ListBuffer

import org.apache.spark.delta.sharing.CachedTableManager
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{DataFrame, DeltaSharingScanUtils, Row, SparkSession, SQLContext}
import org.apache.spark.sql.execution.LogicalRDD
Expand All @@ -26,7 +29,12 @@ import org.apache.spark.sql.functions.col
import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
import org.apache.spark.sql.types.StructType

import io.delta.sharing.spark.model.{CDFColumnInfo, Metadata, Table => DeltaSharingTable}
import io.delta.sharing.spark.model.{
CDFColumnInfo,
DeltaTableFiles,
Metadata,
Table => DeltaSharingTable
}

case class RemoteDeltaCDFRelation(
spark: SparkSession,
Expand All @@ -46,12 +54,25 @@ case class RemoteDeltaCDFRelation(
val metadata = deltaTabelFiles.metadata
val params = RemoteDeltaFileIndexParams(spark, snapshotToUse)
val dfs = ListBuffer[DataFrame]()
val refs = ListBuffer[WeakReference[AnyRef]]()

// We unconditionally add all types of files.
// We will get empty data frames for empty ones, which will get combined later.
dfs.append(scanIndex(new RemoteDeltaCDFAddFileIndex(params, deltaTabelFiles), metadata))
dfs.append(scanIndex(new RemoteDeltaCDCFileIndex(params, deltaTabelFiles), metadata))
dfs.append(scanIndex(new RemoteDeltaCDFRemoveFileIndex(params, deltaTabelFiles), metadata))
val fileIndex1 = RemoteDeltaCDFAddFileIndex(params, deltaTabelFiles)
refs.append(new WeakReference(fileIndex1))
dfs.append(scanIndex(fileIndex1, metadata))

val fileIndex2 = RemoteDeltaCDCFileIndex(params, deltaTabelFiles)
refs.append(new WeakReference(fileIndex2))
dfs.append(scanIndex(fileIndex2, metadata))

val fileIndex3 = RemoteDeltaCDFRemoveFileIndex(params, deltaTabelFiles)
refs.append(new WeakReference(fileIndex3))
dfs.append(scanIndex(fileIndex3, metadata))
CachedTableManager.INSTANCE.register(
params.path.toString, getIdToUrl(deltaTabelFiles), refs, () => {
getIdToUrl(client.getCDFFiles(table, cdfOptions))
})

dfs.reduce((df1, df2) => df1.unionAll(df2))
.select(requiredColumns.map(c => col(quoteIdentifier(c))): _*)
Expand All @@ -75,4 +96,10 @@ case class RemoteDeltaCDFRelation(
val plan = LogicalRelation(relation)
DeltaSharingScanUtils.ofRows(spark, plan)
}

private def getIdToUrl(deltaTabelFiles: DeltaTableFiles): Map[String, String] = {
deltaTabelFiles.addFilesForCdf.map(a => a.id -> a.url).toMap ++
deltaTabelFiles.cdfFiles.map(c => c.id -> c.url).toMap ++
deltaTabelFiles.removeFiles.map(r => r.id -> r.url).toMap
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,6 @@

package io.delta.sharing.spark

import java.lang.ref.WeakReference

import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.spark.delta.sharing.CachedTableManager
import org.apache.spark.sql.SparkSession
Expand Down Expand Up @@ -146,12 +144,6 @@ private[sharing] abstract class RemoteDeltaCDFFileIndexBase(
override def listFiles(
partitionFilters: Seq[Expression],
dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
// Register the files with the pre-signed url fetcher.
CachedTableManager.INSTANCE
.register(params.path.toString, getIdToUrlMap, new WeakReference(this), () => {
getIdToUrlMap
})

// We ignore partition filters for list files, since the server already
// parforms this filtering for CDF.
makePartitionDirectories(actions)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,7 @@ class RemoteSnapshot(
add.id -> add.url
}.toMap
CachedTableManager.INSTANCE
.register(tablePath.toString, idToUrl, new WeakReference(fileIndex), () => {
.register(tablePath.toString, idToUrl, Seq(new WeakReference(fileIndex)), () => {
client.getFiles(table, Nil, None, versionAsOf).files.map { add =>
add.id -> add.url
}.toMap
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,20 +127,20 @@ class CachedTableManager(
* @param tablePath the table path. This is usually the profile file path.
* @param idToUrl the pre signed url map. This will be refreshed when the pre signed urls is going
* to expire.
* @param ref A weak reference which can be used to determine whether the cache is still
* needed. When the weak reference returns null, we will remove the pre signed
* url cache of this table form the cache.
* @param refs A list of weak references which can be used to determine whether the cache is
* still needed. When all the weak references return null, we will remove the pre
* signed url cache of this table form the cache.
* @param refresher A function to re-generate pre signed urls for the table.
*/
def register(
tablePath: String,
idToUrl: Map[String, String],
ref: WeakReference[AnyRef],
refs: Seq[WeakReference[AnyRef]],
refresher: () => Map[String, String]): Unit = {
val cachedTable = new CachedTable(
preSignedUrlExpirationMs + System.currentTimeMillis(),
idToUrl,
Seq(ref),
refs,
System.currentTimeMillis(),
refresher
)
Expand All @@ -158,7 +158,7 @@ class CachedTableManager(
// Overwrite urls with the new registered ones because they are usually newer
oldTable.idToUrl ++ cachedTable.idToUrl,
// Try to avoid storing duplicate references
if (oldTable.refs.exists(_.get eq ref.get)) oldTable.refs else ref +: oldTable.refs,
refs.filterNot(ref => oldTable.refs.exists(_.get eq ref.get)) ++ oldTable.refs,
lastAccess = System.currentTimeMillis(),
refresher
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ class CachedTableManagerSuite extends SparkFunSuite {
manager.register(
"test-table-path",
Map("id1" -> "url1", "id2" -> "url2"),
new WeakReference(ref),
Seq(new WeakReference(ref)),
() => {
Map("id1" -> "url1", "id2" -> "url2")
})
Expand All @@ -46,7 +46,7 @@ class CachedTableManagerSuite extends SparkFunSuite {
manager.register(
"test-table-path2",
Map("id1" -> "url1", "id2" -> "url2"),
new WeakReference(ref),
Seq(new WeakReference(ref)),
() => {
Map("id1" -> "url3", "id2" -> "url4")
})
Expand All @@ -59,7 +59,7 @@ class CachedTableManagerSuite extends SparkFunSuite {
manager.register(
"test-table-path3",
Map("id1" -> "url1", "id2" -> "url2"),
new WeakReference(new AnyRef),
Seq(new WeakReference(new AnyRef)),
() => {
Map("id1" -> "url3", "id2" -> "url4")
})
Expand All @@ -86,7 +86,7 @@ class CachedTableManagerSuite extends SparkFunSuite {
manager.register(
"test-table-path",
Map("id1" -> "url1", "id2" -> "url2"),
new WeakReference(ref),
Seq(new WeakReference(ref)),
() => {
Map("id1" -> "url1", "id2" -> "url2")
})
Expand Down