Skip to content
Closed
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
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,8 @@ public final class UTF8String implements Comparable<UTF8String>, Externalizable,

private static final UTF8String COMMA_UTF8 = UTF8String.fromString(",");
public static final UTF8String EMPTY_UTF8 = UTF8String.fromString("");
public static final UTF8String ZERO_UTF8 = UTF8String.fromString("0");


/**
* Creates an UTF8String from byte array, which should be encoded in UTF-8.
Expand Down Expand Up @@ -1867,4 +1869,21 @@ public void read(Kryo kryo, Input in) {
in.read((byte[]) base);
}

/**
* Convert a long value to its binary format stripping leading zeros.
*/
public static UTF8String toBinaryString(long val) {
int zeros = Long.numberOfLeadingZeros(val);
if (zeros == Long.SIZE) {
return UTF8String.ZERO_UTF8;
} else {
int length = Long.SIZE - zeros;
byte[] bytes = new byte[length];
do {
bytes[--length] = (byte) ((val & 0x1) == 1 ? '1': '0');
val >>>= 1;
} while (length > 0);
return fromBytes(bytes);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1110,4 +1110,21 @@ public void isValid() {
testIsValid("0x9C 0x76 0x17", "0xEF 0xBF 0xBD 0x76 0x17");
}

@Test
public void toBinaryString() {
assertEquals(ZERO_UTF8, UTF8String.toBinaryString(0));
assertEquals(UTF8String.fromString("1"), UTF8String.toBinaryString(1));
assertEquals(UTF8String.fromString("10"), UTF8String.toBinaryString(2));
assertEquals(UTF8String.fromString("100"), UTF8String.toBinaryString(4));
assertEquals(UTF8String.fromString("111"), UTF8String.toBinaryString(7));
assertEquals(
UTF8String.fromString("1111111111111111111111111111111111111111111111111111111111110011"),
UTF8String.toBinaryString(-13));
assertEquals(
UTF8String.fromString("1000000000000000000000000000000000000000000000000000000000000000"),
UTF8String.toBinaryString(Long.MIN_VALUE));
assertEquals(
UTF8String.fromString("111111111111111111111111111111111111111111111111111111111111111"),
UTF8String.toBinaryString(Long.MAX_VALUE));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1008,11 +1008,10 @@ case class Bin(child: Expression)
override def dataType: DataType = SQLConf.get.defaultStringType

protected override def nullSafeEval(input: Any): Any =
UTF8String.fromString(jl.Long.toBinaryString(input.asInstanceOf[Long]))
UTF8String.toBinaryString(input.asInstanceOf[Long])

override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
defineCodeGen(ctx, ev, (c) =>
s"UTF8String.fromString(java.lang.Long.toBinaryString($c))")
defineCodeGen(ctx, ev, c => s"UTF8String.toBinaryString($c)")
}

override protected def withNewChildInternal(newChild: Expression): Bin = copy(child = newChild)
Expand All @@ -1021,7 +1020,6 @@ case class Bin(child: Expression)
object Hex {
private final val hexDigits =
Array[Byte]('0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F')
private final val ZERO_UTF8 = UTF8String.fromBytes(Array[Byte]('0'))

// lookup table to translate '0' -> 0 ... 'F'/'f' -> 15
val unhexDigits = {
Expand Down Expand Up @@ -1053,7 +1051,7 @@ object Hex {

def hex(num: Long): UTF8String = {
val zeros = jl.Long.numberOfLeadingZeros(num)
if (zeros == jl.Long.SIZE) return ZERO_UTF8
if (zeros == jl.Long.SIZE) return UTF8String.ZERO_UTF8
val len = (jl.Long.SIZE - zeros + 3) / 4
var numBuf = num
val value = new Array[Byte](len)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -431,3 +431,31 @@ SELECT conv('-9223372036854775807', 36, 10)
-- !query analysis
Project [conv(-9223372036854775807, 36, 10, true) AS conv(-9223372036854775807, 36, 10)#x]
+- OneRowRelation


-- !query
SELECT BIN(0)
-- !query analysis
Project [bin(cast(0 as bigint)) AS bin(0)#x]
+- OneRowRelation


-- !query
SELECT BIN(25)
-- !query analysis
Project [bin(cast(25 as bigint)) AS bin(25)#x]
+- OneRowRelation


-- !query
SELECT BIN(25L)
-- !query analysis
Project [bin(25) AS bin(25)#x]
+- OneRowRelation


-- !query
SELECT BIN(25.5)
-- !query analysis
Project [bin(cast(25.5 as bigint)) AS bin(25.5)#x]
+- OneRowRelation
Original file line number Diff line number Diff line change
Expand Up @@ -431,3 +431,31 @@ SELECT conv('-9223372036854775807', 36, 10)
-- !query analysis
Project [conv(-9223372036854775807, 36, 10, false) AS conv(-9223372036854775807, 36, 10)#x]
+- OneRowRelation


-- !query
SELECT BIN(0)
-- !query analysis
Project [bin(cast(0 as bigint)) AS bin(0)#x]
+- OneRowRelation


-- !query
SELECT BIN(25)
-- !query analysis
Project [bin(cast(25 as bigint)) AS bin(25)#x]
+- OneRowRelation


-- !query
SELECT BIN(25L)
-- !query analysis
Project [bin(25) AS bin(25)#x]
+- OneRowRelation


-- !query
SELECT BIN(25.5)
-- !query analysis
Project [bin(cast(25.5 as bigint)) AS bin(25.5)#x]
+- OneRowRelation
5 changes: 5 additions & 0 deletions sql/core/src/test/resources/sql-tests/inputs/math.sql
Original file line number Diff line number Diff line change
Expand Up @@ -77,3 +77,8 @@ SELECT conv('9223372036854775808', 10, 16);
SELECT conv('92233720368547758070', 10, 16);
SELECT conv('9223372036854775807', 36, 10);
SELECT conv('-9223372036854775807', 36, 10);

SELECT BIN(0);
SELECT BIN(25);
SELECT BIN(25L);
SELECT BIN(25.5);
32 changes: 32 additions & 0 deletions sql/core/src/test/resources/sql-tests/results/ansi/math.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -797,3 +797,35 @@ org.apache.spark.SparkArithmeticException
"fragment" : "conv('-9223372036854775807', 36, 10)"
} ]
}


-- !query
SELECT BIN(0)
-- !query schema
struct<bin(0):string>
-- !query output
0


-- !query
SELECT BIN(25)
-- !query schema
struct<bin(25):string>
-- !query output
11001


-- !query
SELECT BIN(25L)
-- !query schema
struct<bin(25):string>
-- !query output
11001


-- !query
SELECT BIN(25.5)
-- !query schema
struct<bin(25.5):string>
-- !query output
11001
32 changes: 32 additions & 0 deletions sql/core/src/test/resources/sql-tests/results/math.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -493,3 +493,35 @@ SELECT conv('-9223372036854775807', 36, 10)
struct<conv(-9223372036854775807, 36, 10):string>
-- !query output
18446744073709551615


-- !query
SELECT BIN(0)
-- !query schema
struct<bin(0):string>
-- !query output
0


-- !query
SELECT BIN(25)
-- !query schema
struct<bin(25):string>
-- !query output
11001


-- !query
SELECT BIN(25L)
-- !query schema
struct<bin(25):string>
-- !query output
11001


-- !query
SELECT BIN(25.5)
-- !query schema
struct<bin(25.5):string>
-- !query output
11001