Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
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 @@ -49,13 +49,15 @@
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding;
import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
import org.apache.iceberg.spark.source.SparkTable;
import org.apache.iceberg.transforms.PartitionSpecVisitor;
import org.apache.iceberg.transforms.SortOrderVisitor;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.ByteBuffers;
import org.apache.iceberg.util.Pair;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
Expand Down Expand Up @@ -598,7 +600,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);
byte[] bytes = ByteBuffers.toByteArray((ByteBuffer) lit.value());
return "X'" + BaseEncoding.base16().encode(bytes) + "'";
} else {
return lit.value().toString();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,11 @@ private void assertEquals(String context, Object[] expectedRow, Object[] actualR
Object actualValue = actualRow[col];
if (expectedValue != null && expectedValue.getClass().isArray()) {
String newContext = String.format("%s (nested col %d)", context, col + 1);
assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue);
if (expectedValue instanceof byte[]) {
Assert.assertArrayEquals(newContext, (byte[]) expectedValue, (byte[]) actualValue);
} else {
assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue);
Copy link
Contributor

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[]?

Copy link
Contributor Author

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;

}
} else if (expectedValue != ANY) {
Assert.assertEquals(context + " contents should match", expectedValue, actualValue);
}
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));
}
}