[ https://issues.apache.org/jira/browse/SPARK-40637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
xsys updated SPARK-40637: ------------------------- Description: h3. Describe the bug When we store a BINARY value (e.g. {{BigInt("1").toByteArray)}} / {{{}X'01'{}}}) either via {{spark-shell or spark-sql, and then read it from Spark-shell, it}} outputs {{{}[01]{}}}. However, it does not encode correctly when querying it via {{{}spark-sql{}}}. i.e., Insert via spark-shell, read via spark-shell: display correctly Insert via spark-shell, read via spark-sql: does not display correctly Insert via spark-sql, read via spark-sql: does not display correctly Insert via spark-sql, read via spark-shell: display correctly h3. To Reproduce On Spark 3.2.1 (commit {{{}4f25b3f712{}}}), using {{{}spark-shell{}}}: {code:java} $SPARK_HOME/bin/spark-shell{code} Execute the following: {code:java} scala> import org.apache.spark.sql.Row scala> import org.apache.spark.sql.types._ scala> val rdd = sc.parallelize(Seq(Row(BigInt("1").toByteArray))) rdd: org.apache.spark.rdd.RDD[org.apache.spark.sql.Row] = ParallelCollectionRDD[356] at parallelize at <console>:28 scala> val schema = new StructType().add(StructField("c1", BinaryType, true)) schema: org.apache.spark.sql.types.StructType = StructType(StructField(c1,BinaryType,true)) scala> val df = spark.createDataFrame(rdd, schema) df: org.apache.spark.sql.DataFrame = [c1: binary] scala> df.write.mode("overwrite").format("orc").saveAsTable("binary_vals_shell") scala> spark.sql("select * from binary_vals_shell;").show(false) +----+ |c1 | +----+ |[01]| +----+{code} Then using {{spark-sql}} to (1) query what is inserted via spark-shell to the binary_vals_shell table, and then (2) insert the value via spark-sql to the binary_vals_sql table (we use tee to redirect the log to a file) {code:java} $SPARK_HOME/bin/spark-sql | tee sql.log{code} Execute the following, we only get an empty output in the terminal (but a garbage character in the log file): {code:java} spark-sql> select * from binary_vals_shell; -- query what is inserted via spark-shell; spark-sql> create table binary_vals_sql(c1 BINARY) stored as ORC; spark-sql> insert into binary_vals_sql select X'01'; -- try to insert directly in spark-sql; spark-sql> select * from binary_vals_sql; Time taken: 0.077 seconds, Fetched 1 row(s) {code} >From the log file, we find it shows as a garbage character. (We never >encountered this garbage character in logs of other data types) h3. !image-2022-10-18-12-15-05-576.png! We then return to spark-shell again and run the following: {code:java} scala> spark.sql("select * from binary_vals_sql;").show(false) +----+ |c1 | +----+ |[01]| +----+{code} The binary value does not display correctly via spark-sql, it still displays correctly via spark-shell. h3. Expected behavior We expect the two Spark interfaces ({{{}spark-sql{}}} & {{{}spark-shell{}}}) to behave consistently for the same data type ({{{}BINARY{}}}) & input ({{{}BigInt("1").toByteArray){}}} / {{{}X'01'{}}}) combination. h3. Additional context We also tried Avro and Parquet and encountered the same issue. We believe this is format-independent. was: h3. Describe the bug When we store a BINARY value (e.g. {{BigInt("1").toByteArray)}} / {{{}X'01'{}}}) either via {{spark-shell or spark-sql, and then read it from Spark-shell, it}} outputs {{{}[01]{}}}. However, it does not encode correctly when querying it via {{{}spark-sql{}}}. h3. To Reproduce On Spark 3.2.1 (commit {{{}4f25b3f712{}}}), using {{{}spark-shell{}}}: {code:java} $SPARK_HOME/bin/spark-shell{code} Execute the following: {code:java} scala> import org.apache.spark.sql.Row scala> import org.apache.spark.sql.types._ scala> val rdd = sc.parallelize(Seq(Row(BigInt("1").toByteArray))) rdd: org.apache.spark.rdd.RDD[org.apache.spark.sql.Row] = ParallelCollectionRDD[356] at parallelize at <console>:28 scala> val schema = new StructType().add(StructField("c1", BinaryType, true)) schema: org.apache.spark.sql.types.StructType = StructType(StructField(c1,BinaryType,true)) scala> val df = spark.createDataFrame(rdd, schema) df: org.apache.spark.sql.DataFrame = [c1: binary] scala> df.write.mode("overwrite").format("orc").saveAsTable("binary_vals_shell") scala> spark.sql("select * from binary_vals_shell;").show(false) +----+ |c1 | +----+ |[01]| +----+{code} We can see the output using "spark.sql" in spark-shell. Then using {{spark-sql}} to (1) query what is inserted via spark-shell to the binary_vals_shell table, and then (2) insert the value via spark-sql to the binary_vals_sql table (we use tee to redirect the log to a file) {code:java} $SPARK_HOME/bin/spark-sql | tee sql.log{code} Execute the following, we only get an empty output in the terminal (but a garbage character in the log file): {code:java} spark-sql> select * from binary_vals_shell; -- query what is inserted via spark-shell; spark-sql> create table binary_vals_sql(c1 BINARY) stored as ORC; spark-sql> insert into binary_vals_sql select X'01'; -- try to insert directly in spark-sql; spark-sql> select * from binary_vals_sql; Time taken: 0.077 seconds, Fetched 1 row(s) {code} >From the log file, we find it shows as a garbage character. (We never >encountered this garbage character in logs of other data types) h3. !image-2022-10-18-12-15-05-576.png! We then return to spark-shell again and run the following: {code:java} scala> spark.sql("select * from binary_vals_sql;").show(false) +----+ |c1 | +----+ |[01]| +----+{code} The binary value does not display correctly via spark-sql, it still displays correctly via spark-shell. h3. Expected behavior We expect the two Spark interfaces ({{{}spark-sql{}}} & {{{}spark-shell{}}}) to behave consistently for the same data type ({{{}BINARY{}}}) & input ({{{}BigInt("1").toByteArray){}}} / {{{}X'01'{}}}) combination. h3. Additional context We also tried Avro and Parquet and encountered the same issue. We believe this is format-independent. > Spark-shell can correctly encode BINARY type but Spark-sql cannot > ----------------------------------------------------------------- > > Key: SPARK-40637 > URL: https://issues.apache.org/jira/browse/SPARK-40637 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 3.2.1 > Reporter: xsys > Priority: Major > Attachments: image-2022-10-18-12-15-05-576.png > > > h3. Describe the bug > When we store a BINARY value (e.g. {{BigInt("1").toByteArray)}} / > {{{}X'01'{}}}) either via {{spark-shell or spark-sql, and then read it from > Spark-shell, it}} outputs {{{}[01]{}}}. However, it does not encode correctly > when querying it via {{{}spark-sql{}}}. > i.e., > Insert via spark-shell, read via spark-shell: display correctly > Insert via spark-shell, read via spark-sql: does not display correctly > Insert via spark-sql, read via spark-sql: does not display correctly > Insert via spark-sql, read via spark-shell: display correctly > h3. To Reproduce > On Spark 3.2.1 (commit {{{}4f25b3f712{}}}), using {{{}spark-shell{}}}: > {code:java} > $SPARK_HOME/bin/spark-shell{code} > Execute the following: > {code:java} > scala> import org.apache.spark.sql.Row > scala> import org.apache.spark.sql.types._ > scala> val rdd = sc.parallelize(Seq(Row(BigInt("1").toByteArray))) > rdd: org.apache.spark.rdd.RDD[org.apache.spark.sql.Row] = > ParallelCollectionRDD[356] at parallelize at <console>:28 > scala> val schema = new StructType().add(StructField("c1", BinaryType, true)) > schema: org.apache.spark.sql.types.StructType = > StructType(StructField(c1,BinaryType,true)) > scala> val df = spark.createDataFrame(rdd, schema) > df: org.apache.spark.sql.DataFrame = [c1: binary] > scala> > df.write.mode("overwrite").format("orc").saveAsTable("binary_vals_shell") > scala> spark.sql("select * from binary_vals_shell;").show(false) > +----+ > |c1 | > +----+ > |[01]| > +----+{code} > Then using {{spark-sql}} to (1) query what is inserted via spark-shell to the > binary_vals_shell table, and then (2) insert the value via spark-sql to the > binary_vals_sql table (we use tee to redirect the log to a file) > {code:java} > $SPARK_HOME/bin/spark-sql | tee sql.log{code} > Execute the following, we only get an empty output in the terminal (but a > garbage character in the log file): > {code:java} > spark-sql> select * from binary_vals_shell; -- query what is inserted via > spark-shell; > spark-sql> create table binary_vals_sql(c1 BINARY) stored as ORC; > spark-sql> insert into binary_vals_sql select X'01'; -- try to insert > directly in spark-sql; > spark-sql> select * from binary_vals_sql; > Time taken: 0.077 seconds, Fetched 1 row(s) > {code} > From the log file, we find it shows as a garbage character. (We never > encountered this garbage character in logs of other data types) > h3. !image-2022-10-18-12-15-05-576.png! > We then return to spark-shell again and run the following: > {code:java} > scala> spark.sql("select * from binary_vals_sql;").show(false) > +----+ > > |c1 | > +----+ > |[01]| > +----+{code} > The binary value does not display correctly via spark-sql, it still displays > correctly via spark-shell. > h3. Expected behavior > We expect the two Spark interfaces ({{{}spark-sql{}}} & {{{}spark-shell{}}}) > to behave consistently for the same data type ({{{}BINARY{}}}) & input > ({{{}BigInt("1").toByteArray){}}} / {{{}X'01'{}}}) combination. > > h3. Additional context > We also tried Avro and Parquet and encountered the same issue. We believe > this is format-independent. -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org