Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-6628][SQL][Branch-2.1] Fix ClassCastException when executing sql statement 'insert into' on hbase table #18127

Closed
wants to merge 1 commit into from

Conversation

weiqingy
Copy link
Contributor

What changes were proposed in this pull request?

The issue of SPARK-6628 is:

org.apache.hadoop.hive.hbase.HiveHBaseTableOutputFormat 

cannot be cast to

org.apache.hadoop.hive.ql.io.HiveOutputFormat

The reason is:

public interface HiveOutputFormat<K, V> extends OutputFormat<K, V> {…}

public class HiveHBaseTableOutputFormat extends
    TableOutputFormat<ImmutableBytesWritable> implements
    OutputFormat<ImmutableBytesWritable, Object> {...}

From the two snippets above, we can see both HiveHBaseTableOutputFormat and HiveOutputFormat extends/implements OutputFormat, and can not cast to each other.

For Spark 1.6, 2.0, 2.1, Spark initials the outputFormat in SparkHiveWriterContainer. For Spark 2.2+, Spark initials the outputFormat in HiveFileFormat.

@transient private lazy val outputFormat =
        jobConf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]]

outputFormat above has to be HiveOutputFormat. However, when users insert data into hbase, the outputFormat is HiveHBaseTableOutputFormat, it isn't instance of HiveOutputFormat.

This PR is to make outputFormat to be "null" when the OutputFormat is not an instance of HiveOutputFormat. This change should be safe since outputFormat is only used to get the file extension in function getFileExtension().

We can also submit this PR to Master branch.

How was this patch tested?

Manually test.
(1) create a HBase table with Hive:

CREATE TABLE testwq100 (row_key string COMMENT 'from deserializer', application string COMMENT 'from deserializer', starttime timestamp COMMENT 'from deserializer', endtime timestamp COMMENT 'from deserializer', status string COMMENT 'from deserializer', statusid smallint COMMENT 'from deserializer',   insertdate timestamp COMMENT 'from deserializer', count int COMMENT 'from deserializer', errordesc string COMMENT 'from deserializer') ROW FORMAT SERDE 'org.apache.hadoop.hive.hbase.HBaseSerDe' STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( 'hbase.columns.mapping'='cf1:application,cf1:starttime,cf1:endtime,cf1:Status,cf1:StatusId,cf1:InsertDate,cf1:count,cf1:ErrorDesc', 'line.delim'='\\n',   'mapkey.delim'='\\u0003', 'serialization.format'='\\u0001') TBLPROPERTIES ('transient_lastDdlTime'='1489696241', 'hbase.table.name' = 'xyz', 'hbase.mapred.output.outputtable' = 'xyz')

(2) verify:

Before:

Insert data into the Hbase table testwq100 from Spark SQL:

scala> sql(s"INSERT INTO testwq100 VALUES ('AA1M22','AA1M122','2011722','201156','Starte1d6',45,20,1,'ad1')")
17/05/26 00:09:10 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 3)
java.lang.ClassCastException: org.apache.hadoop.hive.hbase.HiveHBaseTableOutputFormat cannot be cast to org.apache.hadoop.hive.ql.io.HiveOutputFormat
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.outputFormat$lzycompute(hiveWriterContainers.scala:82)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.outputFormat(hiveWriterContainers.scala:81)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.getOutputName(hiveWriterContainers.scala:101)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.initWriters(hiveWriterContainers.scala:125)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.executorSideSetup(hiveWriterContainers.scala:94)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.writeToFile(hiveWriterContainers.scala:182)
	at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:210)
	at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:210)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:99)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
17/05/26 00:09:10 WARN TaskSetManager: Lost task 0.0 in stage 3.0 (TID 3, localhost, executor driver): java.lang.ClassCastException: org.apache.hadoop.hive.hbase.HiveHBaseTableOutputFormat cannot be cast to org.apache.hadoop.hive.ql.io.HiveOutputFormat
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.outputFormat$lzycompute(hiveWriterContainers.scala:82)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.outputFormat(hiveWriterContainers.scala:81)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.getOutputName(hiveWriterContainers.scala:101)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.initWriters(hiveWriterContainers.scala:125)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.executorSideSetup(hiveWriterContainers.scala:94)
	at org.apache.spark.sql.hive.SparkHiveWriterContainer.writeToFile(hiveWriterContainers.scala:182)
	at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:210)
	at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:210)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:99)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)

After:

scala> sql(s"INSERT INTO testwq100 VALUES ('AA1M22','AA1M122','2011722','201156','Starte1d6',45,20,1,'ad1')")
res2: org.apache.spark.sql.DataFrame = []

scala> sql("select * from testwq100").show
+-------+-----------+---------+-------+---------+--------+--------------------+-----+---------+
|row_key|application|starttime|endtime|   status|statusid|          insertdate|count|errordesc|
+-------+-----------+---------+-------+---------+--------+--------------------+-----+---------+
|   AA1M|       AA1M|     null|   null| Starte1d|      45|                null|    1|      ad1|
| AA1M22|    AA1M122|     null|   null|Starte1d6|      45|1970-01-01 00:00:...|    1|      ad1|
+-------+-----------+---------+-------+---------+--------+--------------------+-----+---------+

The ClassCastException gone. "Insert" succeed.

@SparkQA
Copy link

SparkQA commented May 27, 2017

Test build #77449 has finished for PR 18127 at commit 6a622b0.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@weiqingy
Copy link
Contributor Author

Jenkins, test this please.

@SparkQA
Copy link

SparkQA commented May 27, 2017

Test build #77451 has finished for PR 18127 at commit 6a622b0.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

Hi @weiqingy, I just wonder if it is in progress in any way.

@weiqingy
Copy link
Contributor Author

Thanks, @HyukjinKwon . Yes, but will come back here after I finish other work. Do I need to close this for now and reopen it at that time?

@HyukjinKwon
Copy link
Member

Thanks for your input @weiqingy. I was just trying to suggest to close PRs inactive for a month to review comments and/or non-successful Jenkins test result (for a good reason, of course). Would that take longer than a month?

@infoankitp
Copy link
Contributor

Hi @weiqingy

Just wanted to confirm, if this was fixed in Spark 2.4 or not. Since, I am facing same issue when inserting records in a Hive-Hbase Table.

Also, kindly specify the reason behind not including this change in further versions, if there is any. Kindly also let me know if this issue will be fixed in any upcoming release

@lhsvobodaj
Copy link

Hi @weiqingy @HyukjinKwon

I'm also getting the java.lang.ClassCastException: org.apache.hadoop.hive.hbase.HiveHBaseTableOutputFormat cannot be cast to org.apache.hadoop.hive.ql.io.HiveOutputFormat error when trying to write to a Hive external table stored in HBase.

I was checking the code in this pull request, and I couldn't find it merged in any branch/tag. I notice that starting from version 2.2.0 the code has been moved to a class called HiveFileFormat.scala.

Is this pull request necessary to fix the issue? If so, is there any temporary workaround?

@racc
Copy link

racc commented May 15, 2019

@lhsvobodaj @weiqingy @HyukjinKwon

I think there has been a regression in the current codebase, here:

jobConf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]]

@lhsvobodaj
Copy link

@racc

I'm not sure if it was a regression as I couldn't find the fix merged in any branch.

One of the possible reasons to not merge this code is this Hive issue HIVE-20678.
As the Hive team fixed this, I believe there is no need for changes on the Spark side.

@racc
Copy link

racc commented May 17, 2019

@lhsvobodaj ok so that issue is fixed in Hive 4.0.0, but the problem is that we run on Cloudera's Distribution of Hadoop which uses an older version of Hive and there's no way around it then :(

@lhsvobodaj
Copy link

@racc We are also using CDH (5.11.2 and 6.1.0). The Hive fix for this issue is available on CDH 5.15.2 as per its release notes.

@HyukjinKwon
Copy link
Member

Please check if this still exists in the master and open a PR with a test.

@Moh-BOB
Copy link

Moh-BOB commented Jul 15, 2020

Hi Team,
I'm facing the same issue when trying to write to a Hive external table stored in HBase (through pySpark):
java.lang.ClassCastException: org.apache.hadoop.hive.hbase.HiveHBaseTableOutputFormat cannot be cast to org.apache.hadoop.hive.ql.io.HiveOutputFormat

With this configufration (Spark 2.4 / Hive 2.3.7 / Hadoop 2.7 / Hbase 1.4.9), without CDH.
Is there any solution ?
Thanks.

@CaucherWang
Copy link

Hi, Team,
I'm also facing the same issue when insert into a hive-hbase table by a simple test sql.
I don't think this issue has been resolved.
My configuration is that Spark2.4.6/ Hive 2.3.7/ Hadoop2.7/Hbase1.4.13.

@phxism
Copy link

phxism commented Nov 30, 2022

Hi, Team, I'm also facing the same issue when insert into a hive-hbase table by a simple test sql. I don't think this issue has been resolved. My configuration is that Spark2.4.6/ Hive 2.3.7/ Hadoop2.7/Hbase1.4.13.

+1

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

9 participants