Skip to content

Conversation

@shivsood
Copy link
Contributor

@shivsood shivsood commented Nov 15, 2019

This is a port SPARK-29644 to 2.4

What changes were proposed in this pull request?

Corrected ShortType and ByteType mapping to SmallInt and TinyInt, corrected setter methods to set ShortType and ByteType as setShort() and setByte(). Changes in JDBCUtils.scala
Fixed Unit test cases to where applicable and added new E2E test cases in to test table read/write using ShortType and ByteType.

Problems

  • In master in JDBCUtils.scala line number 547 and 551 have a problem where ShortType and ByteType are set as Integers rather than set as Short and Byte respectively.
case ShortType =>
    (stmt: PreparedStatement, row: Row, pos: Int) =>
    stmt.setInt(pos + 1, row.getShort(pos))
    The issue was pointed out by @maropu

case ByteType =>
    (stmt: PreparedStatement, row: Row, pos: Int) =>
     stmt.setInt(pos + 1, row.getByte(pos))
  • Also at line JDBCUtils.scala 247 TinyInt is interpreted wrongly as IntergetType in getCatalystType()
    case java.sql.Types.TINYINT => IntegerType

  • At line 172 ShortType was wrongly interpreted as IntegerType
    case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT))

  • All thru out tests, ShortType and ByteType were being interpreted as IntegerTypes.

Why are the changes needed?

Given type should be set using the right type.

Does this PR introduce any user-facing change?

Yes.

  • User will now be able to create tables where dataframe contains ByteType when using JDBC connector in overwrite mode.
  • Users will see a SQL side table are created with the right data type. ShortType in spark will translate to smallint and ByteType to TinyInt on the SQL side. This will resulting in small size of db tables where applicable.

How was this patch tested?

Corrected Unit test cases where applicable. Validated in CI/CD
Added/fixed test case in MsSqlServerIntegrationSuite.scala, PostgresIntegrationSuite.scala , MySQLIntegrationSuite.scala to
write/read tables from dataframe with cols as shorttype and bytetype. Validated by manual as follows.

./build/mvn install -DskipTests
./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12

What changes were proposed in this pull request?

Corrected ShortType and ByteType mapping to SmallInt and TinyInt, corrected setter methods to set ShortType and ByteType as setShort() and setByte(). Changes in JDBCUtils.scala
Fixed Unit test cases to where applicable and added new E2E test cases in to test table read/write using ShortType and ByteType.

Problems
- In master in JDBCUtils.scala line number 547 and 551 have a problem where ShortType and ByteType are set as Integers rather than set as Short and Byte respectively.

```
case ShortType =>
    (stmt: PreparedStatement, row: Row, pos: Int) =>
    stmt.setInt(pos + 1, row.getShort(pos))
    The issue was pointed out by @maropu

case ByteType =>
    (stmt: PreparedStatement, row: Row, pos: Int) =>
     stmt.setInt(pos + 1, row.getByte(pos))

- Also at line JDBCUtils.scala 247 TinyInt is interpreted wrongly as IntergetType in getCatalystType()
``` case java.sql.Types.TINYINT => IntegerType ```

- At line 172 ShortType was wrongly interpreted as IntegerType
``` case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT)) ```

- All thru out tests, ShortType and ByteType were being interpreted as IntegerTypes.

Why are the changes needed?
Given type should be set using the right type.

Does this PR introduce any user-facing change?
No

How was this patch tested?
Corrected Unit test cases where applicable. Validated in CI/CD
Added/fixed test case in MsSqlServerIntegrationSuite.scala, PostgresIntegrationSuite.scala , MySQLIntegrationSuite.scala to
write/read tables from dataframe with cols as shorttype and bytetype. Validated by manual as follows.
./build/mvn install -DskipTests
./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12
@dongjoon-hyun
Copy link
Member

ok to test

@shivsood
Copy link
Contributor Author

@dongjoon-hyun @maropu @srowen here's the PR that ports #26301 to Spark 2.4. Thanks

@dongjoon-hyun
Copy link
Member

Yup. I've been waiting. (BTW, actually, I expected your follow-up PR on test case refactoring on master branch first.) 😄

@srowen
Copy link
Member

srowen commented Nov 15, 2019

I think this could be OK; it's a behavior change though? is it OK for a 2.4.x maintenance release? I guess it depends on how much we think this is more bug fix than improvement.

@dongjoon-hyun
Copy link
Member

Yes. I agree with you, @srowen . Due to this patch, the user facing behavior will be fixed.

cc @gatorsmile and @cloud-fan

@shivsood
Copy link
Contributor Author

Yup. I've been waiting. (BTW, actually, I expected your follow-up PR on test case refactoring on master branch first.) smile

@dongjoon-hyun That's on my mind. Will take some time as i have to better understand the current suite and my availability next few weeks. But definitely on me to fix

@SparkQA
Copy link

SparkQA commented Nov 15, 2019

Test build #113894 has finished for PR 26549 at commit feca67a.

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

@maropu
Copy link
Member

maropu commented Nov 15, 2019

Does this PR introduce any user-facing change?
No

Can you update the description above?
Anyway, IMO this is not a serious bug fix, so it might be less worth correcting it in this branch in terms of bug compatibility for 2.4.

@shivsood
Copy link
Contributor Author

Can you update the description above?
Yes,

Anyway, IMO this is not a serious bug fix, so it might be less worth correcting it in this branch.
@maropu Originally i found the issue when validating on spark 2.4. I think its important. AFIAK without this fix, i was not able to overwrite tables in SQL server if the dataframe had a ByteType column. This was because Spark sent a JDBC create command using "BYTE" which is not a know JDBC type. The fixes in jdbcutils.scala now fix this.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

+1, LGTM.

srowen pushed a commit that referenced this pull request Nov 17, 2019
…mallInt and TinyInt in JDBCUtils

This is a port SPARK-29644 to 2.4

### What changes were proposed in this pull request?

Corrected ShortType and ByteType mapping to SmallInt and TinyInt, corrected setter methods to set ShortType and ByteType as setShort() and setByte(). Changes in JDBCUtils.scala
Fixed Unit test cases to where applicable and added new E2E test cases in to test table read/write using ShortType and ByteType.

Problems
- In master in JDBCUtils.scala line number 547 and 551 have a problem where ShortType and ByteType are set as Integers rather than set as Short and Byte respectively.

```
case ShortType =>
    (stmt: PreparedStatement, row: Row, pos: Int) =>
    stmt.setInt(pos + 1, row.getShort(pos))
    The issue was pointed out by maropu

case ByteType =>
    (stmt: PreparedStatement, row: Row, pos: Int) =>
     stmt.setInt(pos + 1, row.getByte(pos))
```

- Also at line JDBCUtils.scala 247 TinyInt is interpreted wrongly as IntergetType in getCatalystType()
``` case java.sql.Types.TINYINT => IntegerType ```

- At line 172 ShortType was wrongly interpreted as IntegerType
``` case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT)) ```

- All thru out tests, ShortType and ByteType were being interpreted as IntegerTypes.

### Why are the changes needed?
Given type should be set using the right type.

### Does this PR introduce any user-facing change?
Yes.
- User will now be able to create tables where dataframe contains ByteType when using JDBC connector in overwrite mode.
- Users will see a SQL side table are created with the right data type. ShortType in spark will translate to smallint and ByteType to TinyInt on the SQL side. This will resulting in small size of db tables where applicable.

### How was this patch tested?
Corrected Unit test cases where applicable. Validated in CI/CD
Added/fixed test case in MsSqlServerIntegrationSuite.scala, PostgresIntegrationSuite.scala , MySQLIntegrationSuite.scala to
write/read tables from dataframe with cols as shorttype and bytetype. Validated by manual as follows.
```
./build/mvn install -DskipTests
./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12
```

Closes #26549 from shivsood/port_29644_2.4.

Authored-by: shivsood <shivsood@microsoft.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
@srowen
Copy link
Member

srowen commented Nov 17, 2019

Merged to 2.4

@srowen srowen closed this Nov 17, 2019
@dongjoon-hyun
Copy link
Member

Thank you, @shivsood , @srowen , @maropu

@gatorsmile
Copy link
Member

How serious is it to the end users? I tend to agree @maropu this is not a big issue.

You know, fixing this in the maintenance releases should be avoided. This will change the data type of the external schema. Normally, it will easily break the external applications.

Can we revert this?

|INSERT INTO numbers VALUES (
|0,
|255, 32767, 2147483647, 9223372036854775807,
|127, 32767, 2147483647, 9223372036854775807,
Copy link
Member

Choose a reason for hiding this comment

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

Wait, how was this possible before? Do we cover unsigned cases too?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, all test cases including signed and unsigned are covered now for ByteType and ShortType. Refer to test("SPARK-29644: Write tables with ShortType") and test("SPARK-29644: Write tables with ByteType") in JDBCWriteSuite.scala and MsSQLServerIntegrationSuite.scala.

Earlier it all worked as everything was treated as an integer. Every test cases treated ByteType and ShortType as integers. These test are corrected now.

Copy link
Member

Choose a reason for hiding this comment

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

I meant

  1. This TINYINT seems able to contain unsigned values (https://dev.mysql.com/doc/refman/8.0/en/integer-types.html) up to 255. How do we handle?
  2. Previously the value given here was 255 for TINYINT which is performed via MySQL if I am not mistaken. How was this possible without UNSIGNED keyword?

Copy link
Member

Choose a reason for hiding this comment

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

@HyukjinKwon .
It seems that we need to revert #23400 with the same reason. How do you think about that?

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Nov 18, 2019

Fixing bugs always change the behavior. All existing application which depends on the bug are affected naturally. I think this is a kind of issue of release note or migration guide (not a revert).

@shivsood
Copy link
Contributor Author

Fixing bugs always change the behavior. All existing application which depends on the bug are affected naturally. I think this is a kind of issue of release note or migration guide (not a revert).
Totally agree. Let me know where i can add documentation on corrected behavior. I have summarized in the PR description briefly.

@srowen
Copy link
Member

srowen commented Nov 18, 2019

These are good questions. If this is not just a bug fix, letting things work that didn't before, it maybe shouldn't be in 2.4. That's the question at #26301 - we can discuss there. If there isn't a quick resolution, we can revert this pending further investigation.

@dongjoon-hyun
Copy link
Member

Hi, All.
I'll revert this PR from branch-2.4 due to the ByteType logic and the above discussion.

@dongjoon-hyun
Copy link
Member

@shivsood . Please make a followup PR against master branch to revert ByteType change.

@srowen
Copy link
Member

srowen commented Nov 18, 2019

Agree, these are significant enough questions that we should reevaluate in master first. Good catch @gatorsmile ! thank you.

@shivsood
Copy link
Contributor Author

Thanks everyone for the comments. 'll send a PR to revert the changes in master.
cc @dongjoon-hyun @srowen @gatorsmile @HyukjinKwon @maropu

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Nov 19, 2019

Along with this, I suggest to revert the followings which mapped SQL TINYINT to Spark ByteType.
[SPARK-26499] [SQL] JdbcUtils.makeGetter does not handle ByteType

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Nov 19, 2019

@dongjoon-hyun, seems #23400 itself is fine although the test alone might have a potential issue. If inferred type is a bytetype, it should get a byte. If there's an issue in type inference in JDBC, we should fix that code path instead.

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Nov 19, 2019

+1 for the partial revert. Only the following part is valid although row.update is already replaced by row.setByte. The test should not be there.

    case ByteType =>
       (rs: ResultSet, row: InternalRow, pos: Int) =>
         row.update(pos, rs.getByte(pos + 1))

@dongjoon-hyun
Copy link
Member

@HyukjinKwon and I double-checked that. Luckily, TINYINT of H2 database is -128 to 127. So, the H2 dialect test code seems valid from the context. We don't need any changes.

@HyukjinKwon
Copy link
Member

Thanks for investigation.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants