kbendick commented on a change in pull request #3460:
URL: https://github.com/apache/iceberg/pull/3460#discussion_r742220523
##########
File path: api/src/main/java/org/apache/iceberg/expressions/Literals.java
##########
@@ -639,5 +647,12 @@ Object writeReplace() throws ObjectStreamException {
protected Type.TypeID typeId() {
return Type.TypeID.BINARY;
}
+
+ @Override
+ public String toString() {
+ byte[] binary = new byte[value().remaining()];
+ value().duplicate().get(binary);
Review comment:
Same note as above.
##########
File path:
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java
##########
@@ -39,6 +39,7 @@
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
+import org.junit.internal.ExactComparisonCriteria;
Review comment:
Is it normal to use this class that's got `internal` in the package
name? Do we have to worry about behavioral changes if we upgrade JUnit?
##########
File path:
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -659,17 +658,7 @@ public String or(String leftResult, String rightResult) {
}
private static <T> String
sqlString(List<org.apache.iceberg.expressions.Literal<T>> literals) {
- return
literals.stream().map(DescribeExpressionVisitor::sqlString).collect(Collectors.joining(",
"));
- }
-
- private static String sqlString(org.apache.iceberg.expressions.Literal<?>
lit) {
- if (lit.value() instanceof String) {
- return "'" + lit.value() + "'";
- } else if (lit.value() instanceof ByteBuffer) {
- throw new IllegalArgumentException("Cannot convert bytes to SQL
literal: " + lit);
- } else {
- return lit.value().toString();
- }
+ return
literals.stream().map(Object::toString).collect(Collectors.joining(", "));
Review comment:
This seems possibly incorrect or a deviation from the behavior before.
It looks like this code is calling Object.toString on `Literal<T>`, whereas
before, the `toString` call came from `Literal<T>::value`. Is that the intended
change?
##########
File path:
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -636,17 +635,17 @@ public String or(String leftResult, String rightResult) {
case NOT_NAN:
return "not_nan(" + pred.ref().name() + ")";
case LT:
- return pred.ref().name() + " < " + sqlString(pred.literal());
+ return pred.ref().name() + " < " + pred.literal();
case LT_EQ:
- return pred.ref().name() + " <= " + sqlString(pred.literal());
+ return pred.ref().name() + " <= " + pred.literal();
case GT:
- return pred.ref().name() + " > " + sqlString(pred.literal());
+ return pred.ref().name() + " > " + pred.literal();
case GT_EQ:
- return pred.ref().name() + " >= " + sqlString(pred.literal());
+ return pred.ref().name() + " >= " + pred.literal();
case EQ:
- return pred.ref().name() + " = " + sqlString(pred.literal());
+ return pred.ref().name() + " = " + pred.literal();
case NOT_EQ:
- return pred.ref().name() + " != " + sqlString(pred.literal());
+ return pred.ref().name() + " != " + pred.literal();
Review comment:
Why did we get rid of the `sqlString` function here? Is there a change
that removes the need to quote strings for example?
Also, since the representation of `ByteBuffer` is potentially engine
specific, would it be better to add the conversion to the `sqlString` here for
Spark (e.g. using the leading capital X, like `X'123456'`).
This would allow other engines to handle it themselves (e.g. if Flink
doesn't use leading `X` format).
I worry about unintended side effects from removing the explicit quoting as
I've seen more PRs come up that handle adding and removing quotes as needed,
but possibly you're more familiar with those conversations than I am.
##########
File path: api/src/main/java/org/apache/iceberg/expressions/Literals.java
##########
@@ -639,5 +647,12 @@ Object writeReplace() throws ObjectStreamException {
protected Type.TypeID typeId() {
return Type.TypeID.BINARY;
}
+
+ @Override
+ public String toString() {
+ byte[] binary = new byte[value().remaining()];
+ value().duplicate().get(binary);
+ return "0x" + BaseEncoding.base16().encode(binary);
Review comment:
I'm not sure if we have any other places where we convert a `ByteBufer`
to a string.
If there are any, it seems like potentially we should add the stringify
function to the `ByteBuffer` utility class and then try to use it uniformly
everywhere. You don't necessarily need to update the other places to use them
in this PR, but would you mind taking a look and seeing if you find any?
##########
File path: api/src/main/java/org/apache/iceberg/expressions/Literals.java
##########
@@ -639,5 +647,12 @@ Object writeReplace() throws ObjectStreamException {
protected Type.TypeID typeId() {
return Type.TypeID.BINARY;
}
+
+ @Override
+ public String toString() {
+ byte[] binary = new byte[value().remaining()];
+ value().duplicate().get(binary);
+ return "0x" + BaseEncoding.base16().encode(binary);
Review comment:
Follow up question: Are there cases where people might not use base16
for the representation or where the leading zero might not be used? I don't
think so for Spark, but I'm less sure with Flink for example.
For Spark, looking at the documentation, it seems like it's just `X`,
without the leading zero:
https://spark.apache.org/docs/latest/sql-ref-literals.html#binary-literal
I'm not as sure about Trino, Flink, and other systems though.
For Trino, it looks to also start with the capital `X` (without the zero):
https://trino.io/docs/current/language/types.html#varbinary
Looking it up, for Flink, it looks like maybe it's actually represented as
`BINARY(n)` or `VARBINARY(n)`. This is the one I'm least sure about:
https://nightlies.apache.org/flink/flink-docs-release-1.13/docs/dev/table/types/#binary
##########
File path: api/src/main/java/org/apache/iceberg/expressions/Literals.java
##########
@@ -599,6 +600,13 @@ public String toString() {
Object writeReplace() throws ObjectStreamException {
return new SerializationProxies.FixedLiteralProxy(value());
}
+
+ @Override
+ public String toString() {
+ byte[] binary = new byte[value().remaining()];
+ value().duplicate().get(binary);
Review comment:
There's an existing ByteBuffers utility class,
org.apache.iceberg.util.ByteBuffers, which you can use here. Can you please use
that for consistency? It's also more efficient in some cases (doesn't always
necessarily allocate, handles offsets more thoroughly, etc).
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]