Skip to content

Commit

Permalink
Add support for SQL binary literals
Browse files Browse the repository at this point in the history
This change adds binary literals without support for embedded separators. The type
of a binary literal is VARBINARY.

No spaces are allowed between the X and the single quote. In our grammar, such sequence
parses successfully as a "generic literal". We expect this to be a common mistake, so we
introduce a check to prevent construction of a generic literal with type X and provide
a clarifying error message.

From the SQL spec:

<binary string literal> ::=
    X <quote> [ <space>... ] [ { <hexit> [ <space>... ] <hexit> [ <space>... ] }... ] <quote>
      [ { <separator> <quote> [ <space>... ]
        [ { <hexit> [ <space>... ]  <hexit> [ <space>... ] }... ] <quote> }... ]

<hexit> ::=
    <digit> | A | B | C | D | E | F | a | b | c | d | e | f

Additionally:

1) The sequence

   <quote> [ <space>... ] { <hexit> [ <space>... ] <hexit> [ <space>... ] }... <quote>

   is equivalent to the sequence

   <quote> { <hexit> <hexit> }... <quote>

2) The sequence

   <quote> { <hexit> <hexit> }... <quote> <separator>
   <quote> { <hexit> <hexit> }... <quote>

   is equivalent to the sequence

   <quote> { <hexit> <hexit> }... { <hexit> <hexit> }... <quote>

3) The introductory 'X' may be represented either in upper case (as 'X') or in lower
   case (as 'x').

4) It is implementation-defined whether the declared type of a binary string literal is a
   fixed-length binary string type, a variable-length binary string type, or a binary large
   object string type.
  • Loading branch information
Yang Yang authored and martint committed Dec 21, 2015
1 parent a7d8350 commit eeeef2d
Show file tree
Hide file tree
Showing 18 changed files with 289 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import com.facebook.presto.sql.tree.ArrayConstructor;
import com.facebook.presto.sql.tree.AstVisitor;
import com.facebook.presto.sql.tree.BetweenPredicate;
import com.facebook.presto.sql.tree.BinaryLiteral;
import com.facebook.presto.sql.tree.BooleanLiteral;
import com.facebook.presto.sql.tree.Cast;
import com.facebook.presto.sql.tree.CoalesceExpression;
Expand Down Expand Up @@ -101,6 +102,7 @@
import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP;
import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.AMBIGUOUS_ATTRIBUTE;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.EXPRESSION_NOT_CONSTANT;
Expand Down Expand Up @@ -568,6 +570,13 @@ protected Type visitStringLiteral(StringLiteral node, AnalysisContext context)
return VARCHAR;
}

@Override
protected Type visitBinaryLiteral(BinaryLiteral node, AnalysisContext context)
{
expressionTypes.put(node, VARBINARY);
return VARBINARY;
}

@Override
protected Type visitLongLiteral(LongLiteral node, AnalysisContext context)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import com.facebook.presto.sql.analyzer.SemanticException;
import com.facebook.presto.sql.tree.ArithmeticUnaryExpression;
import com.facebook.presto.sql.tree.AstVisitor;
import com.facebook.presto.sql.tree.BinaryLiteral;
import com.facebook.presto.sql.tree.BooleanLiteral;
import com.facebook.presto.sql.tree.Cast;
import com.facebook.presto.sql.tree.DoubleLiteral;
Expand Down Expand Up @@ -210,6 +211,12 @@ protected Slice visitStringLiteral(StringLiteral node, ConnectorSession session)
return node.getSlice();
}

@Override
protected Slice visitBinaryLiteral(BinaryLiteral node, ConnectorSession session)
{
return node.getValue();
}

@Override
protected Object visitGenericLiteral(GenericLiteral node, ConnectorSession session)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import com.facebook.presto.sql.tree.ArrayConstructor;
import com.facebook.presto.sql.tree.AstVisitor;
import com.facebook.presto.sql.tree.BetweenPredicate;
import com.facebook.presto.sql.tree.BinaryLiteral;
import com.facebook.presto.sql.tree.BooleanLiteral;
import com.facebook.presto.sql.tree.Cast;
import com.facebook.presto.sql.tree.CoalesceExpression;
Expand Down Expand Up @@ -69,6 +70,7 @@
import static com.facebook.presto.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE;
import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.sql.relational.Expressions.call;
import static com.facebook.presto.sql.relational.Expressions.constant;
Expand Down Expand Up @@ -184,6 +186,12 @@ protected RowExpression visitStringLiteral(StringLiteral node, Void context)
return constant(node.getSlice(), VARCHAR);
}

@Override
protected RowExpression visitBinaryLiteral(BinaryLiteral node, Void context)
{
return constant(node.getValue(), VARBINARY);
}

@Override
protected RowExpression visitGenericLiteral(GenericLiteral node, Void context)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.facebook.presto.spi.block.RunLengthEncodedBlock;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.type.ArrayType;
import io.airlift.slice.Slice;

import java.util.ArrayList;
import java.util.Arrays;
Expand All @@ -32,6 +33,7 @@
import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP;
import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.testing.TestingConnectorSession.SESSION;
import static io.airlift.slice.Slices.wrappedIntArray;
Expand Down Expand Up @@ -100,6 +102,28 @@ public static Block createStringsBlock(Iterable<String> values)
return builder.build();
}

public static Block createSlicesBlock(Slice... values)
{
requireNonNull(values, "varargs 'values' is null");
return createSlicesBlock(Arrays.asList(values));
}

public static Block createSlicesBlock(Iterable<Slice> values)
{
BlockBuilder builder = VARBINARY.createBlockBuilder(new BlockBuilderStatus(), 100);

for (Slice value : values) {
if (value == null) {
builder.appendNull();
}
else {
VARBINARY.writeSlice(builder, value);
}
}

return builder.build();
}

public static Block createStringSequenceBlock(int start, int end)
{
BlockBuilder builder = VARCHAR.createBlockBuilder(new BlockBuilderStatus(), 100);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@
import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.UncheckedExecutionException;
import io.airlift.slice.Slice;
import io.airlift.slice.Slices;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;

Expand All @@ -84,6 +85,7 @@
import static com.facebook.presto.block.BlockAssertions.createBooleansBlock;
import static com.facebook.presto.block.BlockAssertions.createDoublesBlock;
import static com.facebook.presto.block.BlockAssertions.createLongsBlock;
import static com.facebook.presto.block.BlockAssertions.createSlicesBlock;
import static com.facebook.presto.block.BlockAssertions.createStringsBlock;
import static com.facebook.presto.block.BlockAssertions.createTimestampsWithTimezoneBlock;
import static com.facebook.presto.metadata.FunctionKind.SCALAR;
Expand All @@ -94,6 +96,7 @@
import static com.facebook.presto.spi.type.DateTimeEncoding.packDateTimeWithZone;
import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.sql.QueryUtil.mangleFieldReference;
import static com.facebook.presto.sql.analyzer.ExpressionAnalyzer.analyzeExpressionsWithSymbols;
Expand Down Expand Up @@ -125,7 +128,9 @@ public final class FunctionAssertions
createLongsBlock(new DateTime(2001, 8, 22, 3, 4, 5, 321, DateTimeZone.UTC).getMillis()),
createStringsBlock("%el%"),
createStringsBlock((String) null),
createTimestampsWithTimezoneBlock(packDateTimeWithZone(new DateTime(1970, 1, 1, 0, 1, 0, 999, DateTimeZone.UTC).getMillis(), TimeZoneKey.getTimeZoneKey("Z"))));
createTimestampsWithTimezoneBlock(packDateTimeWithZone(new DateTime(1970, 1, 1, 0, 1, 0, 999, DateTimeZone.UTC).getMillis(), TimeZoneKey.getTimeZoneKey("Z"))),
createSlicesBlock(Slices.wrappedBuffer((byte) 0xab))
);

private static final Page ZERO_CHANNEL_PAGE = new Page(1);

Expand All @@ -138,6 +143,7 @@ public final class FunctionAssertions
.put(5, VARCHAR)
.put(6, VARCHAR)
.put(7, TIMESTAMP_WITH_TIME_ZONE)
.put(8, VARBINARY)
.build();

private static final Map<Symbol, Integer> INPUT_MAPPING = ImmutableMap.<Symbol, Integer>builder()
Expand All @@ -149,6 +155,7 @@ public final class FunctionAssertions
.put(new Symbol("bound_pattern"), 5)
.put(new Symbol("bound_null_string"), 6)
.put(new Symbol("bound_timestamp_with_timezone"), 7)
.put(new Symbol("bound_binary_literal"), 8)
.build();

private static final Map<Symbol, Type> SYMBOL_TYPES = ImmutableMap.<Symbol, Type>builder()
Expand All @@ -160,6 +167,7 @@ public final class FunctionAssertions
.put(new Symbol("bound_pattern"), VARCHAR)
.put(new Symbol("bound_null_string"), VARCHAR)
.put(new Symbol("bound_timestamp_with_timezone"), TIMESTAMP_WITH_TIME_ZONE)
.put(new Symbol("bound_binary_literal"), VARBINARY)
.build();

private static final PageSourceProvider PAGE_SOURCE_PROVIDER = new TestPageSourceProvider();
Expand Down Expand Up @@ -671,16 +679,18 @@ public ConnectorPageSource createPageSource(Session session, Split split, List<C
assertInstanceOf(split.getConnectorSplit(), FunctionAssertions.TestSplit.class);
FunctionAssertions.TestSplit testSplit = (FunctionAssertions.TestSplit) split.getConnectorSplit();
if (testSplit.isRecordSet()) {
RecordSet records = InMemoryRecordSet.builder(ImmutableList.<Type>of(BIGINT, VARCHAR, DOUBLE, BOOLEAN, BIGINT, VARCHAR, VARCHAR, TIMESTAMP_WITH_TIME_ZONE)).addRow(
1234L,
"hello",
12.34,
true,
new DateTime(2001, 8, 22, 3, 4, 5, 321, DateTimeZone.UTC).getMillis(),
"%el%",
null,
packDateTimeWithZone(new DateTime(1970, 1, 1, 0, 1, 0, 999, DateTimeZone.UTC).getMillis(), TimeZoneKey.getTimeZoneKey("Z"))
).build();
RecordSet records = InMemoryRecordSet.builder(ImmutableList.<Type>of(BIGINT, VARCHAR, DOUBLE, BOOLEAN, BIGINT, VARCHAR, VARCHAR, TIMESTAMP_WITH_TIME_ZONE, VARBINARY))
.addRow(
1234L,
"hello",
12.34,
true,
new DateTime(2001, 8, 22, 3, 4, 5, 321, DateTimeZone.UTC).getMillis(),
"%el%",
null,
packDateTimeWithZone(new DateTime(1970, 1, 1, 0, 1, 0, 999, DateTimeZone.UTC).getMillis(), TimeZoneKey.getTimeZoneKey("Z")),
Slices.wrappedBuffer((byte) 0xab))
.build();
return new RecordPageSource(records);
}
else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,13 @@ public class TestVarbinaryFunctions
}
}

@Test
public void testBinaryLiteral()
throws Exception
{
assertFunction("X'58F7'", VARBINARY, new SqlVarbinary(new byte[]{(byte) 0x58, (byte) 0xF7}));
}

@Test
public void testLength()
throws Exception
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.facebook.presto.spi.RecordCursor;
import com.facebook.presto.spi.type.SqlTimestampWithTimeZone;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.VarbinaryType;
import com.facebook.presto.sql.parser.SqlParser;
import com.facebook.presto.sql.planner.ExpressionInterpreter;
import com.facebook.presto.sql.planner.Symbol;
Expand Down Expand Up @@ -71,6 +72,7 @@ public class TestExpressionInterpreter
private static final Map<Symbol, Type> SYMBOL_TYPES = ImmutableMap.<Symbol, Type>builder()
.put(new Symbol("bound_long"), BIGINT)
.put(new Symbol("bound_string"), VARCHAR)
.put(new Symbol("bound_varbinary"), VarbinaryType.VARBINARY)
.put(new Symbol("bound_double"), DOUBLE)
.put(new Symbol("bound_boolean"), BOOLEAN)
.put(new Symbol("bound_date"), DATE)
Expand Down Expand Up @@ -158,6 +160,9 @@ public void testComparison()
assertOptimizedEquals("bound_long = unbound_long", "1234 = unbound_long");

assertOptimizedEquals("10151082135029368 = 10151082135029369", "false");

assertOptimizedEquals("bound_varbinary = X'a b'", "true");
assertOptimizedEquals("bound_varbinary = X'a d'", "false");
}

@Test
Expand Down Expand Up @@ -962,6 +967,8 @@ public void testLiterals()

optimize("interval '3' day * unbound_long");
optimize("interval '3' year * unbound_long");

assertEquals(optimize("X'1234'"), Slices.wrappedBuffer((byte) 0x12, (byte) 0x34));
}

private static void assertLike(byte[] value, String pattern, boolean expected)
Expand Down Expand Up @@ -1035,6 +1042,8 @@ public Object getValue(Symbol symbol)
return utf8Slice("%el%");
case "bound_timestamp_with_timezone":
return new SqlTimestampWithTimeZone(new DateTime(1970, 1, 1, 1, 0, 0, 999, DateTimeZone.UTC).getMillis(), getTimeZoneKey("Z"));
case "bound_varbinary":
return Slices.wrappedBuffer((byte) 0xab);
}

return new QualifiedNameReference(symbol.toQualifiedName());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.type.SqlTimestamp;
import com.facebook.presto.spi.type.SqlTimestampWithTimeZone;
import com.facebook.presto.spi.type.SqlVarbinary;
import com.facebook.presto.spi.type.TimeZoneKey;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.sql.tree.Extract.Field;
Expand All @@ -39,6 +40,7 @@
import io.airlift.log.Logger;
import io.airlift.log.Logging;
import io.airlift.slice.Slice;
import io.airlift.slice.Slices;
import io.airlift.units.Duration;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
Expand All @@ -64,6 +66,7 @@
import static com.facebook.presto.spi.type.TimeZoneKey.UTC_KEY;
import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP;
import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.type.JsonType.JSON;
import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
Expand Down Expand Up @@ -172,6 +175,8 @@ public void smokedTest()
assertExecute("'foo'", VARCHAR, "foo");
assertExecute("4.2", DOUBLE, 4.2);
assertExecute("1 + 1", BIGINT, 2L);
assertExecute("X' 1 f'", VARBINARY, new SqlVarbinary(Slices.wrappedBuffer((byte) 0x1f).getBytes()));
assertExecute("X' '", VARBINARY, new SqlVarbinary(new byte[0]));
assertExecute("bound_long", BIGINT, 1234L);
assertExecute("bound_string", VARCHAR, "hello");
assertExecute("bound_double", DOUBLE, 12.34);
Expand All @@ -180,6 +185,7 @@ public void smokedTest()
assertExecute("bound_pattern", VARCHAR, "%el%");
assertExecute("bound_null_string", VARCHAR, null);
assertExecute("bound_timestamp_with_timezone", TIMESTAMP_WITH_TIME_ZONE, new SqlTimestampWithTimeZone(new DateTime(1970, 1, 1, 0, 1, 0, 999, DateTimeZone.UTC).getMillis(), TimeZoneKey.getTimeZoneKey("Z")));
assertExecute("bound_binary_literal", VARBINARY, new SqlVarbinary(new byte[]{(byte) 0xab}));

// todo enable when null output type is supported
// assertExecute("null", null);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -242,6 +242,7 @@ primaryExpression
| number #numericLiteral
| booleanValue #booleanLiteral
| STRING #stringLiteral
| BINARY_LITERAL #binaryLiteral
| POSITION '(' valueExpression IN valueExpression ')' #position
| '(' expression (',' expression)+ ')' #rowConstructor
| ROW '(' expression (',' expression)* ')' #rowConstructor
Expand Down Expand Up @@ -540,6 +541,13 @@ STRING
: '\'' ( ~'\'' | '\'\'' )* '\''
;

// Note: we allow any character inside the binary literal and validate
// its a correct literal when the AST is being constructed. This
// allows us to provide more meaningful error messages to the user
BINARY_LITERAL
: 'X\'' (~'\'')* '\''
;

INTEGER_VALUE
: DIGIT+
;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import com.facebook.presto.sql.tree.ArrayConstructor;
import com.facebook.presto.sql.tree.AstVisitor;
import com.facebook.presto.sql.tree.BetweenPredicate;
import com.facebook.presto.sql.tree.BinaryLiteral;
import com.facebook.presto.sql.tree.BooleanLiteral;
import com.facebook.presto.sql.tree.Cast;
import com.facebook.presto.sql.tree.CoalesceExpression;
Expand Down Expand Up @@ -152,6 +153,12 @@ protected String visitStringLiteral(StringLiteral node, Boolean unmangleNames)
return formatStringLiteral(node.getValue());
}

@Override
protected String visitBinaryLiteral(BinaryLiteral node, Boolean unmangleNames)
{
return "X'" + node.toHexString() + "'";
}

@Override
protected String visitArrayConstructor(ArrayConstructor node, Boolean unmangleNames)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.facebook.presto.sql.tree.AllColumns;
import com.facebook.presto.sql.tree.ArithmeticBinaryExpression;
import com.facebook.presto.sql.tree.AstVisitor;
import com.facebook.presto.sql.tree.BinaryLiteral;
import com.facebook.presto.sql.tree.BooleanLiteral;
import com.facebook.presto.sql.tree.ComparisonExpression;
import com.facebook.presto.sql.tree.Cube;
Expand Down Expand Up @@ -248,6 +249,13 @@ protected Void visitStringLiteral(StringLiteral node, Integer indentLevel)
return null;
}

@Override
protected Void visitBinaryLiteral(BinaryLiteral node, Integer indentLevel)
{
print(indentLevel, "Binary[" + node.toHexString() + "]");
return null;
}

@Override
protected Void visitBooleanLiteral(BooleanLiteral node, Integer indentLevel)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.facebook.presto.sql.tree.ArithmeticUnaryExpression;
import com.facebook.presto.sql.tree.ArrayConstructor;
import com.facebook.presto.sql.tree.BetweenPredicate;
import com.facebook.presto.sql.tree.BinaryLiteral;
import com.facebook.presto.sql.tree.BooleanLiteral;
import com.facebook.presto.sql.tree.Cast;
import com.facebook.presto.sql.tree.CoalesceExpression;
Expand Down Expand Up @@ -1075,6 +1076,13 @@ public Node visitStringLiteral(SqlBaseParser.StringLiteralContext context)
return new StringLiteral(getLocation(context), unquote(context.STRING().getText()));
}

@Override
public Node visitBinaryLiteral(SqlBaseParser.BinaryLiteralContext context)
{
String raw = context.BINARY_LITERAL().getText();
return new BinaryLiteral(getLocation(context), unquote(raw.substring(1)));
}

@Override
public Node visitTypeConstructor(SqlBaseParser.TypeConstructorContext context)
{
Expand Down
Loading

0 comments on commit eeeef2d

Please sign in to comment.