Skip to content

[SPARK-28587][SQL] Explicitly cast JDBC partition string literals to timestamp/date#25707

Closed
maropu wants to merge 1 commit intoapache:masterfrom
maropu:SPARK-28587-2
Closed

[SPARK-28587][SQL] Explicitly cast JDBC partition string literals to timestamp/date#25707
maropu wants to merge 1 commit intoapache:masterfrom
maropu:SPARK-28587-2

Conversation

@maropu
Copy link
Member

@maropu maropu commented Sep 6, 2019

What changes were proposed in this pull request?

This pr proposes to add explicit casts for generated JDBC partition string literals. In the current master, that logic depends on implicit casts of datasource DBMSs. For example;

// This assumes we have a relation testdb(t timestamp) in PostgreSQL
scala> val df = spark.read.format("jdbc")
  .option("url", "jdbc:postgresql...")
  .option("dbtable", "testdb")
  .option("partitionColumn", "t")
  .option("lowerBound", "1972-07-04 03:30:00")
  .option("upperBound", "1972-07-27 14:11:05")
  .option("numPartitions", 2)
  .load()

The query above generates "t" < '1972-07-15 20:50:32.5' or "t" is null" and "t" >= '1972-07-15 20:50:32.5' internally for where clauses. Since t is timestamp, the clauses depend on implicit casts of PostgreSQL. The current one looks ok in most databases, but I believe explicit casts are more reasonable.

Why are the changes needed?

To support JDBC partitioning broadly.

Does this PR introduce any user-facing change?

No

How was this patch tested?

Existing tests.

@maropu
Copy link
Member Author

maropu commented Sep 6, 2019

cc: @gatorsmile @MaxGekk @HyukjinKwon

@HyukjinKwon
Copy link
Member

Yea, it looks making sense to me.

@SparkQA
Copy link

SparkQA commented Sep 6, 2019

Test build #110211 has finished for PR 25707 at commit 603a37a.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@MaxGekk
Copy link
Member

MaxGekk commented Sep 6, 2019

In general it looks reasonable but I have these concerns:

  • Currently it could work with target column type STRING, probably. With this PR, we require to support particular type TIMESTAMP by user's DBMS. I would guess this can potentially break user's apps if the remote column has different type but DMBS is able to convert values from strings.
  • Will it work (and how well) if target column type TIMESTAMPTZ (TIMESTAMP WITH TIMEZONE). Currently, the type is implicitly converted from STRING to TIMESTAMPTZ as you mentioned in the description but with the changes STRING -> TIMESTAMP -> TIMESTAMPTZ.

@maropu
Copy link
Member Author

maropu commented Sep 8, 2019

@HyukjinKwon @MaxGekk Thanks for the check, guys!

Currently it could work with target column type STRING, probably. With this PR, we require to support particular type TIMESTAMP by user's DBMS. I would guess this can potentially break user's apps if the remote column has different type but DMBS is able to convert values from strings.

Ur, it looks a nice suggestion... I saw this comment then I re-checked the type name for timestamps in the other databases; https://www.w3resource.com/sql/data-type.php#DATETIME
I was thinking the type name for timestamps is specified by the standard, but it seems this is not correct? In some SQL servers, the name is not TIMESTAMP but DATETIME.... I need more time to consider how to fix it.

Will it work (and how well) if target column type TIMESTAMPTZ (TIMESTAMP WITH TIMEZONE). Currently, the type is implicitly converted from STRING to TIMESTAMPTZ as you mentioned in the description but with the changes STRING -> TIMESTAMP -> TIMESTAMPTZ.

I see. But, does the behaviour difference between string->timestamp->timestampz/string->timestampz causes actual user-facing issues?

@github-actions
Copy link

We're closing this PR because it hasn't been updated in a while.
This isn't a judgement on the merit of the PR in any way. It's just
a way of keeping the PR queue manageable.

If you'd like to revive this PR, please reopen it!

@github-actions github-actions bot added the Stale label Dec 26, 2019
@github-actions github-actions bot closed this Dec 27, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants