Skip to content

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

zifeif2 pushed a commit to zifeif2/spark that referenced this pull request Nov 22, 2025
## What changes were proposed in this pull request?

This PR proposes to close stale PRs, mostly the same instances with apache#18017

Closes apache#14085 - [SPARK-16408][SQL] SparkSQL Added file get Exception: is a directory …
Closes apache#14239 - [SPARK-16593] [CORE] [WIP] Provide a pre-fetch mechanism to accelerate shuffle stage.
Closes apache#14567 - [SPARK-16992][PYSPARK] Python Pep8 formatting and import reorganisation
Closes apache#14579 - [SPARK-16921][PYSPARK] RDD/DataFrame persist()/cache() should return Python context managers
Closes apache#14601 - [SPARK-13979][Core] Killed executor is re spawned without AWS key…
Closes apache#14830 - [SPARK-16992][PYSPARK][DOCS] import sort and autopep8 on Pyspark examples
Closes apache#14963 - [SPARK-16992][PYSPARK] Virtualenv for Pylint and pep8 in lint-python
Closes apache#15227 - [SPARK-17655][SQL]Remove unused variables declarations and definations in a WholeStageCodeGened stage
Closes apache#15240 - [SPARK-17556] [CORE] [SQL] Executor side broadcast for broadcast joins
Closes apache#15405 - [SPARK-15917][CORE] Added support for number of executors in Standalone [WIP]
Closes apache#16099 - [SPARK-18665][SQL] set statement state to "ERROR" after user cancel job
Closes apache#16445 - [SPARK-19043][SQL]Make SparkSQLSessionManager more configurable
Closes apache#16618 - [SPARK-14409][ML][WIP] Add RankingEvaluator
Closes apache#16766 - [SPARK-19426][SQL] Custom coalesce for Dataset
Closes apache#16832 - [SPARK-19490][SQL] ignore case sensitivity when filtering hive partition columns
Closes apache#17052 - [SPARK-19690][SS] Join a streaming DataFrame with a batch DataFrame which has an aggregation may not work
Closes apache#17267 - [SPARK-19926][PYSPARK] Make pyspark exception more user-friendly
Closes apache#17371 - [SPARK-19903][PYSPARK][SS] window operator miss the `watermark` metadata of time column
Closes apache#17401 - [SPARK-18364][YARN] Expose metrics for YarnShuffleService
Closes apache#17519 - [SPARK-15352][Doc] follow-up: add configuration docs for topology-aware block replication
Closes apache#17530 - [SPARK-5158] Access kerberized HDFS from Spark standalone
Closes apache#17854 - [SPARK-20564][Deploy] Reduce massive executor failures when executor count is large (>2000)
Closes apache#17979 - [SPARK-19320][MESOS][WIP]allow specifying a hard limit on number of gpus required in each spark executor when running on mesos
Closes apache#18127 - [SPARK-6628][SQL][Branch-2.1] Fix ClassCastException when executing sql statement 'insert into' on hbase table
Closes apache#18236 - [SPARK-21015] Check field name is not null and empty in GenericRowWit…
Closes apache#18269 - [SPARK-21056][SQL] Use at most one spark job to list files in InMemoryFileIndex
Closes apache#18328 - [SPARK-21121][SQL] Support changing storage level via the spark.sql.inMemoryColumnarStorage.level variable
Closes apache#18354 - [SPARK-18016][SQL][CATALYST][BRANCH-2.1] Code Generation: Constant Pool Limit - Class Splitting
Closes apache#18383 - [SPARK-21167][SS] Set kafka clientId while fetch messages
Closes apache#18414 - [SPARK-21169] [core] Make sure to update application status to RUNNING if executors are accepted and RUNNING after recovery
Closes apache#18432 - resolve com.esotericsoftware.kryo.KryoException
Closes apache#18490 - [SPARK-21269][Core][WIP] Fix FetchFailedException when enable maxReqSizeShuffleToMem and KryoSerializer
Closes apache#18585 - SPARK-21359
Closes apache#18609 - Spark SQL merge small files to big files Update InsertIntoHiveTable.scala

Added:
Closes apache#18308 - [SPARK-21099][Spark Core] INFO Log Message Using Incorrect Executor I…
Closes apache#18599 - [SPARK-21372] spark writes one log file even I set the number of spark_rotate_log to 0
Closes apache#18619 - [SPARK-21397][BUILD]Maven shade plugin adding dependency-reduced-pom.xml to …
Closes apache#18667 - Fix the simpleString used in error messages
Closes apache#18782 - Branch 2.1

Added:
Closes apache#17694 - [SPARK-12717][PYSPARK] Resolving race condition with pyspark broadcasts when using multiple threads

Added:
Closes apache#16456 - [SPARK-18994] clean up the local directories for application in future by annother thread
Closes apache#18683 - [SPARK-21474][CORE] Make number of parallel fetches from a reducer configurable
Closes apache#18690 - [SPARK-21334][CORE] Add metrics reporting service to External Shuffle Server

Added:
Closes apache#18827 - Merge pull request 1 from apache/master

## How was this patch tested?

N/A

Author: hyukjinkwon <[email protected]>

Closes apache#18780 from HyukjinKwon/close-prs.
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