Skip to content

Conversation

@yaooqinn
Copy link
Member

@yaooqinn yaooqinn commented Mar 22, 2021

What changes were proposed in this pull request?

Unsigned types may be used to produce smaller in-memory representations of the data. These types used by frameworks(e.g. hive, pig) using parquet. And parquet will map them to its base types.

see more https://github.com/apache/parquet-format/blob/master/LogicalTypes.md
https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift

  /**
   * An unsigned integer value.
   *
   * The number describes the maximum number of meaningful data bits in
   * the stored value. 8, 16 and 32 bit values are stored using the
   * INT32 physical type.  64 bit values are stored using the INT64
   * physical type.
   *
   */
  UINT_8 = 11;
  UINT_16 = 12;
  UINT_32 = 13;
  UINT_64 = 14;
UInt8-[0:255]
UInt16-[0:65535]
UInt32-[0:4294967295]
UInt64-[0:18446744073709551615]

In this PR, we support read UINT_8 as ShortType, UINT_16 as IntegerType, UINT_32 as LongType to fit their range. Support for UINT_64 will be in another PR.

Why are the changes needed?

better parquet support

Does this PR introduce any user-facing change?

yes, we can read unit[8/16/32] from parquet files

How was this patch tested?

new tests

@github-actions github-actions bot added the SQL label Mar 22, 2021
@yaooqinn yaooqinn changed the title [SPARK-34817][SQL] Read parquet unsigned types that stored as int32 physical type [SPARK-34817][SQL] Read parquet unsigned types that stored as int32 physical type in parquet Mar 22, 2021
@SparkQA
Copy link

SparkQA commented Mar 22, 2021

Test build #136327 has finished for PR 31921 at commit 0c8b6d4.

  • This patch fails Java style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Mar 22, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40909/

@SparkQA
Copy link

SparkQA commented Mar 22, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40909/

@SparkQA
Copy link

SparkQA commented Mar 22, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40913/

@yaooqinn
Copy link
Member Author

cc @HyukjinKwon @cloud-fan @dongjoon-hyun thanks for reviewing

@SparkQA
Copy link

SparkQA commented Mar 22, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40913/

case UINT_8 => typeNotSupported()
case UINT_16 => typeNotSupported()
case UINT_32 => typeNotSupported()
case UINT_32 => LongType
Copy link
Member

Choose a reason for hiding this comment

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

These were explicitly unsupported at #9646 .. per @liancheng's advice (who's also Parquet committer). So I'm less sure if this is something we should support.

Copy link
Member

Choose a reason for hiding this comment

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

But it's very old. Almost 6 years ago lol. @liancheng do you have a different thought now?

Copy link
Member Author

@yaooqinn yaooqinn Mar 22, 2021

Choose a reason for hiding this comment

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

Thanks, @HyukjinKwon,
Yea, I have checked that PR too. There's also a suggestion that we support them.
Lately, Wenchen created https://issues.apache.org/jira/browse/SPARK-34786 for reading uint64. As other unsigned types are not supported too and they are a bit more clear than uint64 which needs a decimal, I raised this PR to collect more opinions.

IMO, for Spark, it is worthwhile to be able to support more storage layer features without breaking our own rules.

Copy link
Contributor

Choose a reason for hiding this comment

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

My hunch is that Spark SQL didn't support unsigned integral types at all back then. As long as we support that now, it's OK to have.

Copy link
Contributor

@cloud-fan cloud-fan Mar 22, 2021

Choose a reason for hiding this comment

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

It's mostly about compatibility. Spark won't have unsigned types, but spark should be able to read existing parquet files written by other systems that support unsigned types.

num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn);
} else if (column.dataType() == DataTypes.LongType) {
// We use LongType to handle UINT32
defColumn.readIntegersAsUnsigned(
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: readUnsighedIntegers

Copy link
Contributor

Choose a reason for hiding this comment

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

can we follow 38fbe56 and check if dictionary encoding also needs update?

Copy link
Member Author

Choose a reason for hiding this comment

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

OK, checking~

Copy link
Member Author

Choose a reason for hiding this comment

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

Looks irrelevant to me

Copy link
Member Author

Choose a reason for hiding this comment

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

I have added the dictionary decoding code path, change the parquet data generator a bit to produce right encoded/plain data

@SparkQA
Copy link

SparkQA commented Mar 22, 2021

Test build #136330 has finished for PR 31921 at commit 8ff3267.

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

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Test build #136451 has finished for PR 31921 at commit 0da5d07.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String

// Write support class for nested groups: ParquetWriter initializes GroupWriteSupport
Copy link
Member Author

Choose a reason for hiding this comment

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

we don't need this anymore, the ExampleParquetWriter meets our needs

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Test build #136452 has finished for PR 31921 at commit efe9c4a.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Test build #136455 has started for PR 31921 at commit 3642f91.

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41035/

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41036/

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41036/

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41039/

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41035/

@yaooqinn
Copy link
Member Author

@cloud-fan @liancheng @HyukjinKwon @maropu please take another look

column.putInt(i, dictionary.decodeToInt(dictionaryIds.getDictId(i)));
}
}
} else if (column.dataType() == DataTypes.LongType) {
Copy link
Contributor

Choose a reason for hiding this comment

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

when will we hit this branch? it's case INT32 not unsigned.

Copy link
Member Author

@yaooqinn yaooqinn Mar 24, 2021

Choose a reason for hiding this comment

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

On Parquet side, for signed and unsigned int (<=32) types they share the same PrimitiveType - INT32. The Unsigned ones are just logical types.

canReadAsIntDecimal(column.dataType())) {
defColumn.readIntegers(
num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn);
} else if (column.dataType() == DataTypes.LongType) {
Copy link
Contributor

Choose a reason for hiding this comment

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

shall we add an extra check to make sure we are reading unsigned values?

Copy link
Member Author

Choose a reason for hiding this comment

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

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41039/

int requiredBytes = total * 4;
ByteBuffer buffer = getBuffer(requiredBytes);
for (int i = 0; i < total; i += 1) {
c.putLong(rowId + i, Integer.toUnsignedLong(buffer.getInt()));
Copy link
Member Author

@yaooqinn yaooqinn Mar 24, 2021

Choose a reason for hiding this comment

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

maybe we can improve here by coverting the buffer.array() to unsigned stuffs, but I am not sure it's faster and how to do that right now.

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Test build #136472 has finished for PR 31921 at commit d9afc79.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41056/

@SparkQA
Copy link

SparkQA commented Mar 24, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41056/

@SparkQA
Copy link

SparkQA commented Mar 25, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41076/

@SparkQA
Copy link

SparkQA commented Mar 25, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41076/

@SparkQA
Copy link

SparkQA commented Mar 25, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41081/

@SparkQA
Copy link

SparkQA commented Mar 25, 2021

Test build #136490 has finished for PR 31921 at commit 02cee4f.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in 8c6748f Mar 25, 2021
@SparkQA
Copy link

SparkQA commented Mar 25, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41081/

@SparkQA
Copy link

SparkQA commented Mar 25, 2021

Test build #136497 has finished for PR 31921 at commit 71496bd.

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

HyukjinKwon pushed a commit that referenced this pull request Dec 12, 2023
…ger types

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

JIRA: https://issues.apache.org/jira/browse/SPARK-43427

Protobuf supports unsigned integer types, including uint32 and uint64. When deserializing protobuf values with fields of these types, `from_protobuf` currently transforms them to the spark types of:

```
uint32 => IntegerType
uint64 => LongType
```

IntegerType and LongType are [signed](https://spark.apache.org/docs/latest/sql-ref-datatypes.html) integer types, so this can lead to confusing results. Namely, if a uint32 value in a stored proto is above 2^31 or a uint64 value is above 2^63, their representation in binary will contain a 1 in the highest bit, which when interpreted as a signed integer will be negative (I.e. overflow). No information is lost, as `IntegerType` and `LongType` contain 32 and 64 bits respectively, however their representation can be confusing.

In this PR, we add an option (`upcast.unsigned.ints`) to allow upcasting unsigned integer types into a larger integer type that can represent them natively, i.e.

```
uint32 => LongType
uint64 => Decimal(20, 0)
```

I added an option so that it doesn't break any existing clients.

**Example of current behavior**

Consider a protobuf message like:

```
syntax = "proto3";

message Test {
  uint64 val = 1;
}
```

If we compile the above and then generate a message with a value for `val` above 2^63:

```
import test_pb2

s = test_pb2.Test()
s.val = 9223372036854775809 # 2**63 + 1
serialized = s.SerializeToString()
print(serialized)
```

This generates the binary representation:

b'\x08\x81\x80\x80\x80\x80\x80\x80\x80\x80\x01'

Then, deserializing this using `from_protobuf`, we can see that it is represented as a negative number. I did this in a notebook so its easier to see, but could reproduce in a scala test as well:

![image](https://github.com/apache/spark/assets/1002986/7144e6a9-3f43-455e-94c3-9065ae88206e)

**Precedent**
I believe that unsigned integer types in parquet are deserialized in a similar manner, i.e. put into a larger type so that the unsigned representation natively fits. https://issues.apache.org/jira/browse/SPARK-34817 and #31921. So an option to get similar behavior would be useful.

### Why are the changes needed?
Improve unsigned integer deserialization behavior.

### Does this PR introduce any user-facing change?
Yes, adds a new option.

### How was this patch tested?
Unit Testing

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #43773 from justaparth/parth/43427-add-option-to-expand-unsigned-integers.

Authored-by: Parth Upadhyay <[email protected]>
Signed-off-by: Hyukjin Kwon <[email protected]>
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.

5 participants