Skip to content

Commit

Permalink
[Sharing] Add timestamp_ntz to Delta Sharing reader feature header (#…
Browse files Browse the repository at this point in the history
…3579)

<!--
Thanks for sending a pull request!  Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md
2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]
Your PR title ...'.
  3. Be sure to keep the PR description updated to reflect all changes.
  4. Please write your PR title to summarize what this PR proposes.
5. If possible, provide a concise example to reproduce the issue for a
faster review.
6. If applicable, include the corresponding issue number in the PR title
and link it in the body.
-->

#### Which Delta project/connector is this regarding?
<!--
Please add the component selected below to the beginning of the pull
request title
For example: [Spark] Title of my pull request
-->

- [ ] Spark
- [ ] Standalone
- [ ] Flink
- [ ] Kernel
- [x] Other (Sharing)

## Description

Add timestamp_ntz to Delta Sharing reader feature header.

## How was this patch tested?

Unit test

## Does this PR introduce _any_ user-facing changes?

<!--
If yes, please clarify the previous behavior and the change this PR
proposes - provide the console output, description and/or an example to
show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change
compared to the released Delta Lake versions or within the unreleased
branches such as master.
If no, write 'No'.
-->
  • Loading branch information
charlenelyu-db committed Aug 20, 2024
1 parent e213023 commit 0afde44
Show file tree
Hide file tree
Showing 4 changed files with 123 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,8 @@ import org.apache.spark.sql.delta.{
DeletionVectorsTableFeature,
DeltaLog,
DeltaParquetFileFormat,
SnapshotDescriptor
SnapshotDescriptor,
TimestampNTZTableFeature
}
import org.apache.spark.sql.delta.actions.{Metadata, Protocol}
import com.google.common.hash.Hashing
Expand All @@ -45,9 +46,17 @@ import org.apache.spark.storage.{BlockId, StorageLevel}
object DeltaSharingUtils extends Logging {

val STREAMING_SUPPORTED_READER_FEATURES: Seq[String] =
Seq(DeletionVectorsTableFeature.name, ColumnMappingTableFeature.name)
Seq(
DeletionVectorsTableFeature.name,
ColumnMappingTableFeature.name,
TimestampNTZTableFeature.name
)
val SUPPORTED_READER_FEATURES: Seq[String] =
Seq(DeletionVectorsTableFeature.name, ColumnMappingTableFeature.name)
Seq(
DeletionVectorsTableFeature.name,
ColumnMappingTableFeature.name,
TimestampNTZTableFeature.name
)

// The prefix will be used for block ids of all blocks that store the delta log in BlockManager.
// It's used to ensure delta sharing queries don't mess up with blocks with other applications.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@

package io.delta.sharing.spark

import java.time.LocalDateTime

import org.apache.spark.sql.delta.DeltaIllegalStateException
import org.apache.spark.sql.delta.DeltaLog
import org.apache.spark.sql.delta.DeltaOptions.{
Expand Down Expand Up @@ -692,6 +694,72 @@ class DeltaFormatSharingSourceSuite
}
}

test("streaming works with timestampNTZ") {
withTempDir { tempDir =>
val deltaTableName = "delta_table_timestampNTZ"
withTable(deltaTableName) {
sql(s"CREATE TABLE $deltaTableName(c1 TIMESTAMP_NTZ) USING DELTA")
val sharedTableName = "shared_table_timestampNTZ"
prepareMockedClientMetadata(deltaTableName, sharedTableName)
val profileFile = prepareProfileFile(tempDir)
val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName"

withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) {
def InsertToDeltaTable(values: String): Unit = {
sql(s"INSERT INTO $deltaTableName VALUES $values")
}

def processAllAvailableInStream(
sourceOptions: Map[String, String],
expectations: StreamAction*): Unit = {
val df = spark.readStream
.format("deltaSharing")
.options(sourceOptions)
.load(tablePath)
.select("c1")

val base = Seq(StartStream(), ProcessAllAvailable())
testStream(df)((base ++ expectations): _*)
}

// Insert at version 1.
InsertToDeltaTable("""('2022-01-01 02:03:04.123456')""")
// Insert at version 2.
InsertToDeltaTable("""('2022-02-02 03:04:05.123456')""")

prepareMockedClientAndFileSystemResult(
deltaTable = deltaTableName,
sharedTable = sharedTableName,
versionAsOf = Some(2L)
)
prepareMockedClientGetTableVersion(deltaTableName, sharedTableName)
processAllAvailableInStream(
Map("responseFormat" -> "delta"),
CheckAnswer(
LocalDateTime.parse("2022-01-01T02:03:04.123456"),
LocalDateTime.parse("2022-02-02T03:04:05.123456")
)
)

prepareMockedClientAndFileSystemResultForStreaming(
deltaTableName,
sharedTableName,
startingVersion = 2,
endingVersion = 2
)
processAllAvailableInStream(
Map(
"responseFormat" -> "delta",
"startingVersion" -> "2"
),
CheckAnswer(LocalDateTime.parse("2022-02-02T03:04:05.123456"))
)
assertBlocksAreCleanedUp()
}
}
}
}

test("startingVersion works") {
withTempDirs { (inputDir, outputDir, checkpointDir) =>
val deltaTableName = "delta_table_startVersion"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.sql.types.{
LongType,
StringType,
StructType,
TimestampNTZType,
TimestampType
}

Expand Down Expand Up @@ -1429,6 +1430,42 @@ trait DeltaSharingDataSourceDeltaSuiteBase
}
}
}

test("DeltaSharingDataSource able to read timestampNTZ table") {
withTempDir { tempDir =>
val deltaTableName = "delta_table_timestampNTZ"
withTable(deltaTableName) {
sql(s"CREATE TABLE $deltaTableName(c1 TIMESTAMP_NTZ) USING DELTA")
sql(s"""INSERT INTO $deltaTableName VALUES ('2022-01-02 03:04:05.123456')""")

val sharedTableName = "shared_table_timestampNTZ"
prepareMockedClientAndFileSystemResult(deltaTableName, sharedTableName)
prepareMockedClientGetTableVersion(deltaTableName, sharedTableName)

def testReadTimestampNTZ(tablePath: String): Unit = {
val expectedSchema: StructType = new StructType()
.add("c1", TimestampNTZType)
assert(
expectedSchema == spark.read
.format("deltaSharing")
.option("responseFormat", "delta")
.load(tablePath)
.schema
)
val sharingDf =
spark.read.format("deltaSharing").option("responseFormat", "delta").load(tablePath)
val deltaDf = spark.read.format("delta").table(deltaTableName)
checkAnswer(sharingDf, deltaDf)
assert(sharingDf.count() > 0)
}

withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) {
val profileFile = prepareProfileFile(tempDir)
testReadTimestampNTZ(s"${profileFile.getCanonicalPath}#share1.default.$sharedTableName")
}
}
}
}
}

class DeltaSharingDataSourceDeltaSuite extends DeltaSharingDataSourceDeltaSuiteBase {}
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,12 @@ private[spark] class TestClientForDeltaFormatSharing(

assert(
responseFormat == DeltaSharingRestClient.RESPONSE_FORMAT_PARQUET ||
(readerFeatures.contains("deletionVectors") && readerFeatures.contains("columnMapping")),
"deletionVectors and columnMapping should be supported in all types of queries."
(
readerFeatures.contains("deletionVectors") &&
readerFeatures.contains("columnMapping") &&
readerFeatures.contains("timestampNtz")
),
"deletionVectors, columnMapping, timestampNtz should be supported in all types of queries."
)

import TestClientForDeltaFormatSharing._
Expand Down

0 comments on commit 0afde44

Please sign in to comment.