commit | 8f7e56c82193d1518b7557a87f65b1b1b0d57792 | [log] [tgz] |
---|---|---|
author | ravowlga123 <ravowlga@gmail.com> | Fri Nov 29 15:14:29 2019 +0100 |
committer | Balazs Meszaros <meszibalu@apache.org> | Fri Nov 29 15:14:29 2019 +0100 |
tree | 03468c52fe1e7419ecae813c9d05f6858a52599b | |
parent | 73063c42f409509bd85bf51faf4f449ee02048a3 [diff] |
HBASE-23348 Spark's createTable method throws an exception while the table is being split (#50) Signed-off-by: Guangxu Cheng <gxcheng@apache.org>
diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala index 70f5cab..cede78a 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
@@ -164,7 +164,7 @@ // Initialize hBase table if necessary val admin = connection.getAdmin try { - if (!admin.isTableAvailable(tName)) { + if (!admin.tableExists(tName)) { val tableDesc = new HTableDescriptor(tName) cfs.foreach { x => val cf = new HColumnDescriptor(x.getBytes())