-
Notifications
You must be signed in to change notification settings - Fork 2.9k
Spark: Generate Spark's SQL literal format for binary and update the toString representation of binary/fixed literals. #3728
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
Conversation
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java
Outdated
Show resolved
Hide resolved
| } else if (value instanceof scala.collection.Map) { | ||
| return row.getJavaMap(pos); | ||
| } else if (value.getClass().isArray() && value.getClass().getComponentType().isPrimitive()) { | ||
| return IntStream.range(0, Array.getLength(value)).mapToObj(i -> Array.get(value, i)).toArray(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Question: This change to the test class seems somewhat complicated relative to the other cases. Is there any way to check for value instanceof scala.collection.Array (assuming that Scala types are returned) like in the map case and in the basic list case?
I guess my question is, why do we need this block when we have the check for scala.collection.Seq already? Can you help me understand?
And possibly, since it is a bit more complicated, a comment addressing what case this is expected to handle in the code might be called for.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeap, i’ll simplify the code and add instructions
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What does this convert the byte array to? Why does it need to change? I thought byte[] was fine to return.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The assertEquals method cannot verify whether binary arrays are equal. If we do not convert binary arrays to Object arrays, we may need to create a separate method to compare binary arrays
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This representation should not change for tests. I think that the original type is correct. If you need to fix tests that are failing because of array comparison, then you should update the assertions instead of runtime code.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
okey
| return "0x" + BaseEncoding.base16().encode((toByteBuffer().array())); | ||
| default: | ||
| return String.valueOf(value); | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This method should not change. It is the base implementation. Instead, you should override the implementation in BinaryLiteral and FixedLiteral.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok, thanks for your reply
| sql("CREATE TABLE %s (id bigint, data string, float float) USING iceberg", tableName); | ||
| sql("INSERT INTO %s VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', float('NaN'))", tableName); | ||
| sql("CREATE TABLE %s (id bigint, data string, float float, binary binary) USING iceberg", tableName); | ||
| sql("INSERT INTO %s VALUES (1, 'a', 1.0, X''), (2, 'b', 2.0, X'1111'), (3, 'c', float('NaN'), X'11')", tableName); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please don't edit existing test cases. Instead, add new test cases/methods for what you are changing.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok, thanks for your comment
|
@xiaotianzhang01 can you fix the description and summary for this? This is not adding any parsing to Iceberg, so the title is misleading. Instead, it producing Spark's SQL literal format for binary and updating the toString representation of binary/fixed literals. |
d7f5c4d to
a9f01e6
Compare
fad6688 to
e837b37
Compare
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java
Outdated
Show resolved
Hide resolved
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java
Outdated
Show resolved
Hide resolved
|
@rdblue Thank you very much for your review! I have addressed your comments |
|
|
||
| @Override | ||
| public String toString() { | ||
| return "X'" + BaseEncoding.base16().encode((toByteBuffer().array())) + "'"; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is not safe to use ByteBuffer.array like this. Can you please use ByteBuffers.toByteArray?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
okey, update
…oString representation of binary/fixed literals.
49a8716 to
9ed6de3
Compare
| } else if (lit.value() instanceof ByteBuffer) { | ||
| throw new IllegalArgumentException("Cannot convert bytes to SQL literal: " + lit); | ||
| // produce a literal value that Spark can parse | ||
| return lit.toString(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is fine to update BinaryLiteral.toString and FixedLiteral.toString, but this should not assume that the toString representation of a literal can be parsed by Spark. Instead, this should produce the correct form here, just like it does for String literals.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we implement BinaryLiteral.toString and FixedLiteral.toString, then we don't need to do it separately for ByteBuffer
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, we should not rely on toString using a representation that works for Spark. Instead, this should convert the literal for Spark here. We may change the toString representation in Literal and we don't want this to depend on that method. It's fine for numbers, but there are different representations for binary that are equally valid. This is where we customize the one that Spark uses.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Okay, got it
3d284bc to
b2b8273
Compare
| if (expectedValue instanceof byte[]) { | ||
| Assert.assertArrayEquals(newContext, (byte[]) expectedValue, (byte[]) actualValue); | ||
| } else { | ||
| assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why does the assertEquals with Object[] not work for byte[]?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The SQL query returns byte[] instead of Byte[]. The encapsulated type Byte[] can be converted to Object[], but the conversion of the basic type byte[] will report an error, [B cannot be cast to [Ljava.lang.Object;
|
Thanks, @xiaotianzhang01! Nice work. |
|
Nice work. |
adds the ability to assert binary arrays for "assertEquals" and adds iceberg hexadecimal text parsing
solve cannot convert bytes to SQL literal
cc/ @aokolnychyi @rdblue