Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 14 additions & 0 deletions api/src/main/java/org/apache/iceberg/expressions/Literals.java
Original file line number Diff line number Diff line change
Expand Up @@ -35,10 +35,12 @@
import java.util.Objects;
import java.util.UUID;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding;
import org.apache.iceberg.types.Comparators;
import org.apache.iceberg.types.Conversions;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.ByteBuffers;
import org.apache.iceberg.util.NaNUtil;

class Literals {
Expand Down Expand Up @@ -599,6 +601,12 @@ protected Type.TypeID typeId() {
Object writeReplace() throws ObjectStreamException {
return new SerializationProxies.FixedLiteralProxy(value());
}

@Override
public String toString() {
byte[] bytes = ByteBuffers.toByteArray(value());
return "X'" + BaseEncoding.base16().encode(bytes) + "'";
}
}

static class BinaryLiteral extends BaseLiteral<ByteBuffer> {
Expand Down Expand Up @@ -639,5 +647,11 @@ Object writeReplace() throws ObjectStreamException {
protected Type.TypeID typeId() {
return Type.TypeID.BINARY;
}

@Override
public String toString() {
byte[] bytes = ByteBuffers.toByteArray(value());
return "X'" + BaseEncoding.base16().encode(bytes) + "'";
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -598,7 +598,8 @@ 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);
// produce a literal value that Spark can parse
return lit.toString();
Copy link
Contributor

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.

Copy link
Contributor Author

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

Copy link
Contributor

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Okay, got it

} else {
return lit.value().toString();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.apache.iceberg.spark;

import java.lang.reflect.Array;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -118,6 +119,8 @@ private Object[] toJava(Row row) {
return row.getList(pos);
} else if (value instanceof scala.collection.Map) {
return row.getJavaMap(pos);
} else if (value instanceof byte[]) {
return IntStream.range(0, Array.getLength(value)).mapToObj(i -> Array.get(value, i)).toArray();
Copy link
Contributor

@kbendick kbendick Dec 15, 2021

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.

Copy link
Contributor Author

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

Copy link
Contributor

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.

Copy link
Contributor Author

@Timzhang01 Timzhang01 Jan 6, 2022

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

Copy link
Contributor

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

okey

} else {
return value;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
public class TestSelect extends SparkCatalogTestBase {
private int scanEventCount = 0;
private ScanEvent lastScanEvent = null;
private String binaryTableName = tableName("binary_table");

public TestSelect(String catalogName, String implementation, Map<String, String> config) {
super(catalogName, implementation, config);
Expand All @@ -63,6 +64,7 @@ public void createTables() {
@After
public void removeTables() {
sql("DROP TABLE IF EXISTS %s", tableName);
sql("DROP TABLE IF EXISTS %s", binaryTableName);
}

@Test
Expand Down Expand Up @@ -203,4 +205,14 @@ public void testSpecifySnapshotAndTimestamp() {
.collectAsList();
});
}

@Test
public void testBinaryInFilter() {
sql("CREATE TABLE %s (id bigint, binary binary) USING iceberg", binaryTableName);
sql("INSERT INTO %s VALUES (1, X''), (2, X'1111'), (3, X'11')", binaryTableName);
List<Object[]> expected = ImmutableList.of(row(2L, new Byte[]{0x11, 0x11}));

assertEquals("Should return all expected rows", expected,
sql("SELECT id, binary FROM %s where binary > X'11'", binaryTableName));
}
}